From 2706808095ba8400f6019ed1bf3ab3440fa29bff Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Tue, 10 Jun 2025 18:13:30 +0800 Subject: [PATCH 01/57] feat: add NestedStructSchemaAdapter for handling schema evolution of nested structs --- datafusion/core/src/datasource/mod.rs | 22 +- datafusion/datasource/src/mod.rs | 1 + .../datasource/src/nested_schema_adapter.rs | 943 ++++++++++++++++++ 3 files changed, 956 insertions(+), 10 deletions(-) create mode 100644 datafusion/datasource/src/nested_schema_adapter.rs diff --git a/datafusion/core/src/datasource/mod.rs b/datafusion/core/src/datasource/mod.rs index b3d69064ff15b..00fb8c1aa6026 100644 --- a/datafusion/core/src/datasource/mod.rs +++ b/datafusion/core/src/datasource/mod.rs @@ -42,6 +42,7 @@ pub use datafusion_catalog::default_table_source; pub use datafusion_catalog::memory; pub use datafusion_catalog::stream; pub use datafusion_catalog::view; +pub use datafusion_datasource::nested_schema_adapter; pub use datafusion_datasource::schema_adapter; pub use datafusion_datasource::sink; pub use datafusion_datasource::source; @@ -51,11 +52,8 @@ pub use datafusion_physical_expr::create_ordering; #[cfg(all(test, feature = "parquet"))] mod tests { - use datafusion_datasource::schema_adapter::{ - DefaultSchemaAdapterFactory, SchemaAdapter, SchemaAdapterFactory, SchemaMapper, - }; - use crate::prelude::SessionContext; + use ::object_store::{path::Path, ObjectMeta}; use arrow::{ array::{Int32Array, StringArray}, datatypes::{DataType, Field, Schema, SchemaRef}, @@ -63,13 +61,17 @@ mod tests { }; use datafusion_common::{record_batch, test_util::batches_to_sort_string}; use datafusion_datasource::{ - file::FileSource, file_scan_config::FileScanConfigBuilder, - source::DataSourceExec, PartitionedFile, + file::FileSource, + file_scan_config::FileScanConfigBuilder, + schema_adapter::{ + DefaultSchemaAdapterFactory, SchemaAdapter, SchemaAdapterFactory, + SchemaMapper, + }, + source::DataSourceExec, + PartitionedFile, }; 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; @@ -79,6 +81,7 @@ mod tests { // 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(); @@ -122,8 +125,7 @@ mod tests { let schema = Arc::new(Schema::new(vec![f1.clone(), f2.clone()])); let source = ParquetSource::default() - .with_schema_adapter_factory(Arc::new(TestSchemaAdapterFactory {})) - .unwrap(); + .with_schema_adapter_factory(Arc::new(TestSchemaAdapterFactory {})); let base_conf = FileScanConfigBuilder::new( ObjectStoreUrl::local_filesystem(), schema, diff --git a/datafusion/datasource/src/mod.rs b/datafusion/datasource/src/mod.rs index c79efd11fcc5e..c99e570ea1b84 100644 --- a/datafusion/datasource/src/mod.rs +++ b/datafusion/datasource/src/mod.rs @@ -38,6 +38,7 @@ pub mod file_scan_config; pub mod file_sink_config; pub mod file_stream; pub mod memory; +pub mod nested_schema_adapter; pub mod schema_adapter; pub mod sink; pub mod source; diff --git a/datafusion/datasource/src/nested_schema_adapter.rs b/datafusion/datasource/src/nested_schema_adapter.rs new file mode 100644 index 0000000000000..4af80b1c061e1 --- /dev/null +++ b/datafusion/datasource/src/nested_schema_adapter.rs @@ -0,0 +1,943 @@ +// 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. + +//! [`NestedStructSchemaAdapter`] and [`NestedStructSchemaAdapterFactory`] to adapt file-level record batches to a table schema. +//! +//! Adapter provides a method of translating the RecordBatches that come out of the +//! physical format into how they should be used by DataFusion. For instance, a schema +//! can be stored external to a parquet file that maps parquet logical types to arrow types. + +use arrow::datatypes::{DataType::Struct, Field, Schema, SchemaRef}; +use datafusion_common::{ColumnStatistics, Result}; +use std::sync::Arc; + +use crate::schema_adapter::{ + create_field_mapping, DefaultSchemaAdapterFactory, SchemaAdapter, + SchemaAdapterFactory, SchemaMapper, +}; +use arrow::array::{Array, ArrayRef, StructArray}; +use arrow::compute::cast; +use arrow::record_batch::{RecordBatch, RecordBatchOptions}; +use datafusion_common::arrow::array::new_null_array; + +/// Factory for creating [`NestedStructSchemaAdapter`] +/// +/// This factory creates schema adapters that properly handle schema evolution +/// for nested struct fields, allowing new fields to be added to struct columns +/// over time. +#[derive(Debug, Clone, Default)] +pub struct NestedStructSchemaAdapterFactory; + +impl SchemaAdapterFactory for NestedStructSchemaAdapterFactory { + fn create( + &self, + projected_table_schema: SchemaRef, + table_schema: SchemaRef, + ) -> Box { + Box::new(NestedStructSchemaAdapter::new( + projected_table_schema, + table_schema, + )) + } +} + +impl NestedStructSchemaAdapterFactory { + /// Create a new factory for mapping batches from a file schema to a table + /// schema with support for nested struct evolution. + /// + /// This is a convenience method that handles nested struct fields properly. + pub fn from_schema(table_schema: SchemaRef) -> Box { + Self.create(Arc::clone(&table_schema), table_schema) + } + + /// Determines if a schema contains nested struct fields that would benefit + /// from special handling during schema evolution + pub fn has_nested_structs(schema: &Schema) -> bool { + schema + .fields() + .iter() + .any(|field| matches!(field.data_type(), Struct(_))) + } + + /// Create an appropriate schema adapter based on schema characteristics. + /// Returns a NestedStructSchemaAdapter if the projected schema contains nested structs, + /// otherwise returns a DefaultSchemaAdapter. + pub fn create_adapter( + projected_table_schema: SchemaRef, + table_schema: SchemaRef, + ) -> Box { + // Use nested adapter if target has nested structs + if Self::has_nested_structs(table_schema.as_ref()) { + NestedStructSchemaAdapterFactory.create(projected_table_schema, table_schema) + } else { + // Default case for simple schemas + DefaultSchemaAdapterFactory.create(projected_table_schema, table_schema) + } + } +} + +/// A SchemaAdapter that handles schema evolution for nested struct types +#[derive(Debug, Clone)] +pub struct NestedStructSchemaAdapter { + /// The schema for the table, projected to include only the fields being output (projected) by the + /// associated ParquetSource + projected_table_schema: SchemaRef, + /// The entire table schema for the table we're using this to adapt. + /// + /// This is used to evaluate any filters pushed down into the scan + /// which may refer to columns that are not referred to anywhere + /// else in the plan. + table_schema: SchemaRef, +} + +impl NestedStructSchemaAdapter { + /// Create a new NestedStructSchemaAdapter with the target schema + pub fn new(projected_table_schema: SchemaRef, table_schema: SchemaRef) -> Self { + Self { + projected_table_schema, + table_schema, + } + } + + pub fn projected_table_schema(&self) -> &Schema { + self.projected_table_schema.as_ref() + } + + pub fn table_schema(&self) -> &Schema { + self.table_schema.as_ref() + } +} + +impl SchemaAdapter for NestedStructSchemaAdapter { + fn map_column_index(&self, index: usize, file_schema: &Schema) -> Option { + let field_name = self.table_schema.field(index).name(); + file_schema.index_of(field_name).ok() + } + + fn map_schema( + &self, + file_schema: &Schema, + ) -> Result<(Arc, Vec)> { + let (field_mappings, projection) = create_field_mapping( + file_schema, + &self.projected_table_schema, + |file_field, table_field| { + // Special handling for struct fields - always include them even if the + // internal structure differs, as we'll adapt them later + match (file_field.data_type(), table_field.data_type()) { + (Struct(_), Struct(_)) => Ok(true), + _ => { + // For non-struct fields, use the regular cast check + crate::schema_adapter::can_cast_field(file_field, table_field) + } + } + }, + )?; + + Ok(( + Arc::new(NestedStructSchemaMapping::new( + Arc::clone(&self.projected_table_schema), + field_mappings, + )), + projection, + )) + } +} + +/// A SchemaMapping implementation specifically for nested structs +#[derive(Debug)] +struct NestedStructSchemaMapping { + /// The schema for the table, projected to include only the fields being output + projected_table_schema: SchemaRef, + /// Field mappings from projected table to file schema + field_mappings: Vec>, +} + +impl NestedStructSchemaMapping { + /// Create a new nested struct schema mapping + pub fn new( + projected_table_schema: SchemaRef, + field_mappings: Vec>, + ) -> Self { + Self { + projected_table_schema, + field_mappings, + } + } +} + +/// Maps a `RecordBatch` to a new `RecordBatch` according to the schema mapping defined in `NestedStructSchemaMapping`. +/// +/// # Arguments +/// +/// * `batch` - The input `RecordBatch` to be mapped. +/// +/// # Returns +/// +/// A `Result` containing the new `RecordBatch` with columns adapted according to the schema mapping, or an error if the mapping fails. +/// +/// # Behavior +/// +/// - For each field in the projected table schema, the corresponding column in the input batch is adapted. +/// - If a field does not exist in the input batch, a null array of the appropriate data type and length is created and used in the output batch. +/// - If a field exists in the input batch, the column is adapted to handle potential nested struct adaptation. +/// +/// # Errors +/// +/// Returns an error if the column adaptation fails or if the new `RecordBatch` cannot be created. +impl SchemaMapper for NestedStructSchemaMapping { + fn map_batch(&self, batch: RecordBatch) -> Result { + let batch_rows = batch.num_rows(); + let batch_cols = batch.columns().to_vec(); + + let cols = self + .projected_table_schema + .fields() + .iter() + .zip(&self.field_mappings) + .map(|(field, file_idx)| { + file_idx.map_or_else( + // If field doesn't exist in file, return null array + || Ok(new_null_array(field.data_type(), batch_rows)), + // If field exists, handle potential nested struct adaptation + |batch_idx| adapt_column(&batch_cols[batch_idx], field), + ) + }) + .collect::, _>>()?; + + // Create record batch with adapted columns + let options = RecordBatchOptions::new().with_row_count(Some(batch.num_rows())); + let schema = Arc::clone(&self.projected_table_schema); + let record_batch = RecordBatch::try_new_with_options(schema, cols, &options)?; + Ok(record_batch) + } + + /// Adapts file-level column `Statistics` to match the `table_schema` + /// + /// Maps statistics from the file schema to the projected table schema using field mappings. + /// For fields not present in the file schema, uses unknown statistics. + fn map_column_statistics( + &self, + file_col_statistics: &[ColumnStatistics], + ) -> Result> { + let mut table_col_statistics = vec![]; + + // Map statistics for each field based on field_mappings + for (_, file_col_idx) in self + .projected_table_schema + .fields() + .iter() + .zip(&self.field_mappings) + { + if let Some(file_col_idx) = file_col_idx { + // Use statistics from file if available, otherwise default + table_col_statistics.push( + file_col_statistics + .get(*file_col_idx) + .cloned() + .unwrap_or_default(), + ); + } else { + // Field doesn't exist in file schema, use unknown statistics + table_col_statistics.push(ColumnStatistics::new_unknown()); + } + } + + Ok(table_col_statistics) + } +} + +// Helper methods for the NestedStructSchemaMapping +/// Adapt a column to match the target field type, handling nested structs specially +fn adapt_column(source_col: &ArrayRef, target_field: &Field) -> Result { + match target_field.data_type() { + Struct(target_fields) => { + // For struct arrays, we need to handle them specially + if let Some(struct_array) = source_col.as_any().downcast_ref::() + { + // Create a vector to store field-array pairs with the correct type + let mut children: Vec<(Arc, Arc)> = Vec::new(); + let num_rows = source_col.len(); + + // For each field in the target schema + for target_child_field in target_fields { + // Create Arc directly (not Arc>) + let field_arc = Arc::clone(target_child_field); + + // Try to find corresponding field in source + match struct_array.column_by_name(target_child_field.name()) { + Some(source_child_col) => { + // Field exists in source, adapt it + let adapted_child = + adapt_column(source_child_col, target_child_field)?; + children.push((field_arc, adapted_child)); + } + None => { + // Field doesn't exist in source, add null array + children.push(( + field_arc, + new_null_array(target_child_field.data_type(), num_rows), + )); + } + } + } + + // Create new struct array with all target fields + let struct_array = StructArray::from(children); + Ok(Arc::new(struct_array)) + } else { + // Not a struct array, but target expects struct - return nulls + Ok(new_null_array(target_field.data_type(), source_col.len())) + } + } + // For non-struct types, just cast + _ => Ok(cast(source_col, target_field.data_type())?), + } +} + +#[cfg(test)] +mod tests { + use super::*; + use arrow::array::{Array, StringBuilder, StructArray, TimestampMillisecondArray}; + use arrow::datatypes::{ + DataType::{Float64, Int16, Int32, Timestamp, Utf8}, + TimeUnit::Millisecond, + }; + use datafusion_common::ScalarValue; + + // ================================ + // Schema Creation Helper Functions + // ================================ + + /// Helper function to create a basic nested schema with additionalInfo + fn create_basic_nested_schema() -> SchemaRef { + Arc::new(Schema::new(vec![ + create_additional_info_field(false), // without reason field + ])) + } + + /// Helper function to create a deeply nested schema with additionalInfo including reason field + fn create_deep_nested_schema() -> SchemaRef { + Arc::new(Schema::new(vec![ + create_additional_info_field(true), // with reason field + ])) + } + + /// Helper function to create the additionalInfo field with or without the reason subfield + fn create_additional_info_field(with_reason: bool) -> Field { + let mut field_children = vec![ + Field::new("location", Utf8, true), + Field::new("timestamp_utc", Timestamp(Millisecond, None), true), + ]; + + // Add the reason field if requested (for target schema) + if with_reason { + field_children.push(create_reason_field()); + } + + Field::new("additionalInfo", Struct(field_children.into()), true) + } + + /// Helper function to create the reason nested field with its details subfield + fn create_reason_field() -> Field { + Field::new( + "reason", + Struct( + vec![ + Field::new("_level", Float64, true), + // Inline the details field creation + Field::new( + "details", + Struct( + vec![ + Field::new("rurl", Utf8, true), + Field::new("s", Float64, true), + Field::new("t", Utf8, true), + ] + .into(), + ), + true, + ), + ] + .into(), + ), + true, + ) + } + + // ================================ + // Schema Evolution Tests + // ================================ + + #[test] + fn test_adapter_factory_selection() -> Result<()> { + // Test schemas for adapter selection logic + let simple_schema = Arc::new(Schema::new(vec![ + Field::new("id", Int32, false), + Field::new("name", Utf8, true), + Field::new("age", Int16, true), + ])); + + let nested_schema = Arc::new(Schema::new(vec![ + Field::new("id", Int32, false), + Field::new( + "metadata", + Struct( + vec![ + Field::new("created", Utf8, true), + Field::new("modified", Utf8, true), + ] + .into(), + ), + true, + ), + ])); + + // Source schema with missing field + let source_schema = Arc::new(Schema::new(vec![ + Field::new("id", Int32, false), + Field::new( + "metadata", + Struct( + vec![ + Field::new("created", Utf8, true), + // "modified" field is missing + ] + .into(), + ), + true, + ), + ])); + + // Test struct detection + assert!( + !NestedStructSchemaAdapterFactory::has_nested_structs(&simple_schema), + "Simple schema should not be detected as having nested structs" + ); + assert!( + NestedStructSchemaAdapterFactory::has_nested_structs(&nested_schema), + "Nested schema should be detected as having nested structs" + ); + + // Test adapter behavior with schema evolution + let default_adapter = DefaultSchemaAdapterFactory + .create(nested_schema.clone(), nested_schema.clone()); + let nested_adapter = NestedStructSchemaAdapterFactory + .create(nested_schema.clone(), nested_schema.clone()); + + // Default adapter should fail with schema evolution + assert!(default_adapter.map_schema(&source_schema).is_err()); + + // Nested adapter should handle schema evolution + assert!( + nested_adapter.map_schema(&source_schema).is_ok(), + "Nested adapter should handle schema with missing fields" + ); + + // Test factory selection logic + let adapter = NestedStructSchemaAdapterFactory::create_adapter( + nested_schema.clone(), + nested_schema.clone(), + ); + + assert!( + adapter.map_schema(&source_schema).is_ok(), + "Factory should select appropriate adapter that handles schema evolution" + ); + + Ok(()) + } + + #[test] + fn test_adapt_struct_with_added_nested_fields() -> Result<()> { + // Create test schemas + let (file_schema, table_schema) = create_test_schemas_with_nested_fields(); + + // Create batch with test data + let batch = create_test_batch_with_struct_data(&file_schema)?; + + // Create adapter and apply it + let mapped_batch = + adapt_batch_with_nested_schema_adapter(&file_schema, &table_schema, batch)?; + + // Verify the results + verify_adapted_batch_with_nested_fields(&mapped_batch, &table_schema)?; + + Ok(()) + } + + /// Create file and table schemas for testing nested field evolution + fn create_test_schemas_with_nested_fields() -> (SchemaRef, SchemaRef) { + // Create file schema with just location and timestamp_utc + let file_schema = Arc::new(Schema::new(vec![Field::new( + "info", + Struct( + vec![ + Field::new("location", Utf8, true), + Field::new( + "timestamp_utc", + Timestamp(Millisecond, Some("UTC".into())), + true, + ), + ] + .into(), + ), + true, + )])); + + // Create table schema with additional nested reason field + let table_schema = Arc::new(Schema::new(vec![Field::new( + "info", + Struct( + vec![ + Field::new("location", Utf8, true), + Field::new( + "timestamp_utc", + Timestamp(Millisecond, Some("UTC".into())), + true, + ), + Field::new( + "reason", + Struct( + vec![ + Field::new("_level", Float64, true), + Field::new( + "details", + Struct( + vec![ + Field::new("rurl", Utf8, true), + Field::new("s", Float64, true), + Field::new("t", Utf8, true), + ] + .into(), + ), + true, + ), + ] + .into(), + ), + true, + ), + ] + .into(), + ), + true, + )])); + + (file_schema, table_schema) + } + + /// Create a test RecordBatch with struct data matching the file schema + fn create_test_batch_with_struct_data( + file_schema: &SchemaRef, + ) -> Result { + let mut location_builder = StringBuilder::new(); + location_builder.append_value("San Francisco"); + location_builder.append_value("New York"); + + // Create timestamp array + let timestamp_array = TimestampMillisecondArray::from(vec![ + Some(1640995200000), // 2022-01-01 + Some(1641081600000), // 2022-01-02 + ]); + + // Create data type with UTC timezone to match the schema + let timestamp_type = Timestamp(Millisecond, Some("UTC".into())); + + // Cast the timestamp array to include the timezone metadata + let timestamp_array = cast(×tamp_array, ×tamp_type)?; + + let info_struct = StructArray::from(vec![ + ( + Arc::new(Field::new("location", Utf8, true)), + Arc::new(location_builder.finish()) as Arc, + ), + ( + Arc::new(Field::new("timestamp_utc", timestamp_type, true)), + timestamp_array, + ), + ]); + + Ok(RecordBatch::try_new( + Arc::clone(file_schema), + vec![Arc::new(info_struct)], + )?) + } + + /// Apply the nested schema adapter to the batch + fn adapt_batch_with_nested_schema_adapter( + file_schema: &SchemaRef, + table_schema: &SchemaRef, + batch: RecordBatch, + ) -> Result { + let adapter = NestedStructSchemaAdapter::new( + Arc::clone(table_schema), + Arc::clone(table_schema), + ); + + let (mapper, _) = adapter.map_schema(file_schema.as_ref())?; + mapper.map_batch(batch) + } + + /// Verify the adapted batch has the expected structure and data + fn verify_adapted_batch_with_nested_fields( + mapped_batch: &RecordBatch, + table_schema: &SchemaRef, + ) -> Result<()> { + // Verify the mapped batch structure and data + assert_eq!(mapped_batch.schema(), *table_schema); + assert_eq!(mapped_batch.num_rows(), 2); + + // Extract and verify the info struct column + let info_col = mapped_batch.column(0); + let info_array = info_col + .as_any() + .downcast_ref::() + .expect("Expected info column to be a StructArray"); + + // Verify the original fields are preserved + verify_preserved_fields(info_array)?; + + // Verify the reason field exists with correct structure + verify_reason_field_structure(info_array)?; + + Ok(()) + } + + /// Verify the original fields from file schema are preserved in the adapted batch + fn verify_preserved_fields(info_array: &StructArray) -> Result<()> { + // Verify location field + let location_col = info_array + .column_by_name("location") + .expect("Expected location field in struct"); + let location_array = location_col + .as_any() + .downcast_ref::() + .expect("Expected location to be a StringArray"); + + // Verify the location values are preserved + assert_eq!(location_array.value(0), "San Francisco"); + assert_eq!(location_array.value(1), "New York"); + + // Verify timestamp field + let timestamp_col = info_array + .column_by_name("timestamp_utc") + .expect("Expected timestamp_utc field in struct"); + let timestamp_array = timestamp_col + .as_any() + .downcast_ref::() + .expect("Expected timestamp_utc to be a TimestampMillisecondArray"); + + assert_eq!(timestamp_array.value(0), 1640995200000); + assert_eq!(timestamp_array.value(1), 1641081600000); + + Ok(()) + } + + /// Verify the added reason field structure and null values + fn verify_reason_field_structure(info_array: &StructArray) -> Result<()> { + // Verify the reason field exists and is null + let reason_col = info_array + .column_by_name("reason") + .expect("Expected reason field in struct"); + let reason_array = reason_col + .as_any() + .downcast_ref::() + .expect("Expected reason to be a StructArray"); + + // Verify reason has correct structure + assert_eq!(reason_array.fields().len(), 2); + assert!(reason_array.column_by_name("_level").is_some()); + assert!(reason_array.column_by_name("details").is_some()); + + // Verify details field has correct nested structure + let details_col = reason_array + .column_by_name("details") + .expect("Expected details field in reason struct"); + let details_array = details_col + .as_any() + .downcast_ref::() + .expect("Expected details to be a StructArray"); + + assert_eq!(details_array.fields().len(), 3); + assert!(details_array.column_by_name("rurl").is_some()); + assert!(details_array.column_by_name("s").is_some()); + assert!(details_array.column_by_name("t").is_some()); + + // Verify all added fields are null + for i in 0..2 { + assert!(reason_array.is_null(i), "reason field should be null"); + } + + Ok(()) + } + + // ================================ + // Data Mapping Tests + // ================================ + + // Helper function to verify column statistics match expected values + fn verify_column_statistics( + stats: &ColumnStatistics, + expected_null_count: Option, + expected_distinct_count: Option, + expected_min: Option, + expected_max: Option, + expected_sum: Option, + ) { + if let Some(count) = expected_null_count { + assert_eq!( + stats.null_count, + datafusion_common::stats::Precision::Exact(count), + "Null count should match expected value" + ); + } + + if let Some(count) = expected_distinct_count { + assert_eq!( + stats.distinct_count, + datafusion_common::stats::Precision::Exact(count), + "Distinct count should match expected value" + ); + } + + if let Some(min) = expected_min { + assert_eq!( + stats.min_value, + datafusion_common::stats::Precision::Exact(min), + "Min value should match expected value" + ); + } + + if let Some(max) = expected_max { + assert_eq!( + stats.max_value, + datafusion_common::stats::Precision::Exact(max), + "Max value should match expected value" + ); + } + + if let Some(sum) = expected_sum { + assert_eq!( + stats.sum_value, + datafusion_common::stats::Precision::Exact(sum), + "Sum value should match expected value" + ); + } + } + + // Helper to create test column statistics + fn create_test_column_statistics( + null_count: usize, + distinct_count: usize, + min_value: Option, + max_value: Option, + sum_value: Option, + ) -> ColumnStatistics { + ColumnStatistics { + null_count: datafusion_common::stats::Precision::Exact(null_count), + distinct_count: datafusion_common::stats::Precision::Exact(distinct_count), + min_value: min_value.map_or_else( + || datafusion_common::stats::Precision::Absent, + datafusion_common::stats::Precision::Exact, + ), + max_value: max_value.map_or_else( + || datafusion_common::stats::Precision::Absent, + datafusion_common::stats::Precision::Exact, + ), + sum_value: sum_value.map_or_else( + || datafusion_common::stats::Precision::Absent, + datafusion_common::stats::Precision::Exact, + ), + } + } + + #[test] + fn test_map_column_statistics_basic() -> Result<()> { + // Test statistics mapping with a simple schema + let file_schema = create_basic_nested_schema(); + let table_schema = create_deep_nested_schema(); + + let adapter = NestedStructSchemaAdapter::new( + Arc::clone(&table_schema), + Arc::clone(&table_schema), + ); + + let (mapper, _) = adapter.map_schema(file_schema.as_ref())?; + + // Create test statistics for additionalInfo column + let file_stats = vec![create_test_column_statistics( + 5, + 100, + Some(ScalarValue::Utf8(Some("min_value".to_string()))), + Some(ScalarValue::Utf8(Some("max_value".to_string()))), + Some(ScalarValue::Utf8(Some("sum_value".to_string()))), + )]; + + // Map statistics + let table_stats = mapper.map_column_statistics(&file_stats)?; + + // Verify count and content + assert_eq!( + table_stats.len(), + 1, + "Should have stats for one struct column" + ); + verify_column_statistics( + &table_stats[0], + Some(5), + Some(100), + Some(ScalarValue::Utf8(Some("min_value".to_string()))), + Some(ScalarValue::Utf8(Some("max_value".to_string()))), + Some(ScalarValue::Utf8(Some("sum_value".to_string()))), + ); + + Ok(()) + } + + #[test] + fn test_map_column_statistics_empty() -> Result<()> { + // Test statistics mapping with empty input + let file_schema = create_basic_nested_schema(); + let table_schema = create_deep_nested_schema(); + + let adapter = NestedStructSchemaAdapter::new( + Arc::clone(&table_schema), + Arc::clone(&table_schema), + ); + + let (mapper, _) = adapter.map_schema(file_schema.as_ref())?; + + // Test with missing statistics + let empty_stats = vec![]; + let mapped_empty_stats = mapper.map_column_statistics(&empty_stats)?; + + assert_eq!( + mapped_empty_stats.len(), + 1, + "Should have stats for one column even with empty input" + ); + + assert_eq!( + mapped_empty_stats[0], + ColumnStatistics::new_unknown(), + "Empty input should result in unknown statistics" + ); + + Ok(()) + } + + #[test] + fn test_map_column_statistics_multiple_columns() -> Result<()> { + // Create schemas with multiple columns + let file_schema = Arc::new(Schema::new(vec![ + Field::new("id", Int32, false), + Field::new( + "additionalInfo", + Struct( + vec![ + Field::new("location", Utf8, true), + Field::new( + "timestamp_utc", + Timestamp(Millisecond, Some("UTC".into())), + true, + ), + ] + .into(), + ), + true, + ), + ])); + + let table_schema = Arc::new(Schema::new(vec![ + Field::new("id", Int32, false), + Field::new( + "additionalInfo", + Struct( + vec![ + Field::new("location", Utf8, true), + Field::new( + "timestamp_utc", + Timestamp(Millisecond, Some("UTC".into())), + true, + ), + Field::new( + "reason", + Struct(vec![Field::new("_level", Float64, true)].into()), + true, + ), + ] + .into(), + ), + true, + ), + Field::new("status", Utf8, true), // Extra column in table schema + ])); + + // Create adapter and mapping + let adapter = NestedStructSchemaAdapter::new( + Arc::clone(&table_schema), + Arc::clone(&table_schema), + ); + + let (mapper, _) = adapter.map_schema(file_schema.as_ref())?; + + // Create file column statistics + let file_stats = vec![ + create_test_column_statistics( + 0, + 100, + Some(ScalarValue::Int32(Some(1))), + Some(ScalarValue::Int32(Some(100))), + Some(ScalarValue::Int32(Some(5100))), + ), + create_test_column_statistics(10, 50, None, None, None), + ]; + + // Map statistics + let table_stats = mapper.map_column_statistics(&file_stats)?; + + // Verify mapped statistics + assert_eq!( + table_stats.len(), + 3, + "Should have stats for all 3 columns in table schema" + ); + + // Verify ID column stats + verify_column_statistics( + &table_stats[0], + Some(0), + Some(100), + Some(ScalarValue::Int32(Some(1))), + Some(ScalarValue::Int32(Some(100))), + Some(ScalarValue::Int32(Some(5100))), + ); + + // Verify additionalInfo column stats + verify_column_statistics(&table_stats[1], Some(10), Some(50), None, None, None); + + // Verify status column has unknown stats + assert_eq!( + table_stats[2], + ColumnStatistics::new_unknown(), + "Missing column should have unknown statistics" + ); + + Ok(()) + } +} From dba76b0579ba760188f383b8eec77e10eae4e84e Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Tue, 10 Jun 2025 18:22:43 +0800 Subject: [PATCH 02/57] Fix: unwrap result of schema adapter factory in ParquetSource configuration --- 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 00fb8c1aa6026..c7aef465b6ca4 100644 --- a/datafusion/core/src/datasource/mod.rs +++ b/datafusion/core/src/datasource/mod.rs @@ -125,7 +125,8 @@ mod tests { let schema = Arc::new(Schema::new(vec![f1.clone(), f2.clone()])); let source = ParquetSource::default() - .with_schema_adapter_factory(Arc::new(TestSchemaAdapterFactory {})); + .with_schema_adapter_factory(Arc::new(TestSchemaAdapterFactory {})) + .unwrap(); let base_conf = FileScanConfigBuilder::new( ObjectStoreUrl::local_filesystem(), schema, From 3b7223a9dbe69661b616f230fd94a7f4b24ad67a Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Tue, 10 Jun 2025 18:44:45 +0800 Subject: [PATCH 03/57] feat: add example for handling schema evolution with nested structs in DataFusion --- .../examples/nested_struct2.rs | 331 ++++++++++++++++++ 1 file changed, 331 insertions(+) create mode 100644 datafusion-examples/examples/nested_struct2.rs diff --git a/datafusion-examples/examples/nested_struct2.rs b/datafusion-examples/examples/nested_struct2.rs new file mode 100644 index 0000000000000..25ae53f6671cc --- /dev/null +++ b/datafusion-examples/examples/nested_struct2.rs @@ -0,0 +1,331 @@ +// 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 datafusion::prelude::*; +use datafusion::{ + arrow::{ + array::Array, array::Float64Array, array::StringArray, array::StructArray, + array::TimestampMillisecondArray, datatypes::DataType, datatypes::Field, + datatypes::Schema, datatypes::TimeUnit, record_batch::RecordBatch, + }, + dataframe::DataFrameWriteOptions, + datasource::{ + file_format::parquet::ParquetFormat, + listing::ListingOptions, + listing::ListingTable, + listing::ListingTableConfig, + listing::ListingTableUrl, + // schema_adapter::SchemaAdapterFactory, + }, +}; +use std::{error::Error, fs, sync::Arc}; + +/// Helper function to create a RecordBatch from a Schema and log the process +async fn create_and_write_parquet_file( + ctx: &SessionContext, + schema: &Arc, + schema_name: &str, + file_path: &str, +) -> Result<(), Box> { + let batch = create_batch(schema, schema_name)?; + + let _ = fs::remove_file(file_path); + + let df = ctx.read_batch(batch)?; + + df.write_parquet( + file_path, + DataFrameWriteOptions::default() + .with_single_file_output(true) + .with_sort_by(vec![col("timestamp_utc").sort(true, true)]), + None, + ) + .await?; + + Ok(()) +} + +/// Helper function to create a ListingTableConfig for given paths and schema +async fn create_listing_table_config( + ctx: &SessionContext, + paths: impl Iterator, + schema: &Arc, +) -> Result> { + let config = ListingTableConfig::new_with_multi_paths( + paths + .map(|p| ListingTableUrl::parse(&p)) + .collect::, _>>()?, + ) + .with_schema(schema.as_ref().clone().into()); + + let config = config.infer(&ctx.state()).await?; + + let updated_options = ListingOptions { + file_sort_order: vec![vec![col("timestamp_utc").sort(true, true)]], + ..config + .options + .unwrap_or_else(|| ListingOptions::new(Arc::new(ParquetFormat::default()))) + }; + + let config = config.with_listing_options(updated_options); + + Ok(config) +} + +/// Helper function to create a ListingTable from paths and schema +async fn create_listing_table( + ctx: &SessionContext, + paths: impl Iterator, + schema: &Arc, +) -> Result, Box> { + // Create the config + let config = create_listing_table_config(ctx, paths, schema).await?; + + // Create the listing table + let listing_table = ListingTable::try_new(config)?; + + Ok(Arc::new(listing_table)) +} + +/// Helper function to register a table and execute a query +async fn execute_and_display_query( + ctx: &SessionContext, + table_name: &str, + listing_table: Arc, +) -> Result<(), Box> { + println!("==> {}", table_name); + ctx.register_table(table_name, listing_table)?; + + // Derive query automatically based on table name + let query = format!("SELECT * FROM {} ORDER BY body", table_name); + let df = ctx.sql(&query).await?; + + let _results = df.clone().collect().await?; + // Print the results + df.show().await?; + + Ok(()) +} + +async fn test_datafusion_schema_evolution() -> Result<(), Box> { + let ctx = SessionContext::new(); + + // Create schemas + let schema1 = create_schema1(); + let schema2 = create_schema2(); + let schema3 = create_schema3(); + let schema4 = create_schema4(); + + // Define file paths in an array for easier management + let test_files = ["jobs1.parquet", "jobs2.parquet", "jobs3.parquet"]; + let [path1, path2, path3] = test_files; // Destructure for individual access + + // Create and write parquet files for each schema + create_and_write_parquet_file(&ctx, &schema1, "schema1", path1).await?; + create_and_write_parquet_file(&ctx, &schema2, "schema2", path2).await?; + create_and_write_parquet_file(&ctx, &schema3, "schema3", path3).await?; + + let paths = vec![path1.to_string(), path2.to_string(), path3.to_string()].into_iter(); + let paths2 = vec![path1.to_string(), path2.to_string(), path3.to_string()] + .into_iter() + .rev(); + + // Use the helper function to create the listing tables with different paths + let table_paths = create_listing_table(&ctx, paths, &schema4).await?; + let table_paths2 = create_listing_table(&ctx, paths2, &schema4).await?; + + // Execute query on the first table with table name "paths" + execute_and_display_query( + &ctx, + "paths", // First table with original path order + table_paths, + ) + .await?; + + // Execute query on the second table with table name "paths2" + execute_and_display_query( + &ctx, + "paths_reversed", // Second table with reversed path order + table_paths2, + ) + .await?; + + Ok(()) +} + +fn create_batch( + schema: &Arc, + schema_name: &str, +) -> Result> { + // Create arrays for each field in the schema + let columns = schema + .fields() + .iter() + .map(|field| create_array_for_field(field, 1, schema_name)) + .collect::, _>>()?; + + // Create record batch with the generated arrays + RecordBatch::try_new(schema.clone(), columns).map_err(|e| e.into()) +} + +/// Creates an appropriate array for a given field with the specified length +fn create_array_for_field( + field: &Field, + length: usize, + schema_name: &str, +) -> Result, Box> { + match field.data_type() { + DataType::Utf8 => { + // For the body field, use schema_name; for other fields use default + if field.name() == "body" { + Ok(Arc::new(StringArray::from(vec![Some(schema_name); length]))) + } else { + let default_value = format!("{}_{}", field.name(), 1); + Ok(Arc::new(StringArray::from(vec![ + Some(default_value); + length + ]))) + } + } + DataType::Float64 => { + // Default float value + Ok(Arc::new(Float64Array::from(vec![Some(1.0); length]))) + } + DataType::Timestamp(TimeUnit::Millisecond, tz) => { + // Default timestamp (2021-12-31T12:00:00Z) + let array = + TimestampMillisecondArray::from(vec![Some(1640995200000); length]); + // Create the array with the same timezone as specified in the field + Ok(Arc::new(array.with_data_type(DataType::Timestamp( + TimeUnit::Millisecond, + tz.clone(), + )))) + } + DataType::Struct(fields) => { + // Create arrays for each field in the struct + let struct_arrays = fields + .iter() + .map(|f| { + let array = create_array_for_field(f, length, schema_name)?; + Ok((f.clone(), array)) + }) + .collect::, Box>>()?; + + Ok(Arc::new(StructArray::from(struct_arrays))) + } + _ => Err(format!("Unsupported data type: {}", field.data_type()).into()), + } +} + +fn create_schema1() -> Arc { + let schema1 = Arc::new(Schema::new(vec![ + Field::new("body", DataType::Utf8, true), + Field::new( + "timestamp_utc", + DataType::Timestamp(TimeUnit::Millisecond, Some("UTC".into())), + true, + ), + ])); + schema1 +} + +/// Creates a schema with basic HTTP request fields plus a query_params struct field +fn create_schema2() -> Arc { + // Get the base schema from create_schema1 + let schema1 = create_schema1(); + + // Create a new vector of fields from schema1 + let mut fields = schema1 + .fields() + .iter() + .map(|f| f.as_ref().clone()) + .collect::>(); + + // Add the query_params field + fields.push(Field::new( + "query_params", + DataType::Struct(vec![Field::new("customer_id", DataType::Utf8, true)].into()), + true, + )); + + // Create a new schema with the extended fields + Arc::new(Schema::new(fields)) +} + +/// Creates a schema with HTTP request fields, query_params struct field, and an error field +fn create_schema3() -> Arc { + // Get the base schema from create_schema2 + let schema2 = create_schema2(); + + // Convert to a vector of fields + let mut fields = schema2 + .fields() + .iter() + .map(|f| f.as_ref().clone()) + .collect::>(); + + // Add the error field + fields.push(Field::new("error", DataType::Utf8, true)); + + // Create a new schema with the extended fields + Arc::new(Schema::new(fields)) +} + +/// Creates a schema with HTTP request fields, expanded query_params struct with additional fields, and an error field +fn create_schema4() -> Arc { + // Get the base schema from create_schema1 (we can't use schema3 directly since we need to modify query_params) + let schema1 = create_schema1(); + + // Convert to a vector of fields + let mut fields = schema1 + .fields() + .iter() + .map(|f| f.as_ref().clone()) + .collect::>(); + + // Add the expanded query_params field with additional fields + fields.push(Field::new( + "query_params", + DataType::Struct( + vec![ + Field::new("customer_id", DataType::Utf8, true), + Field::new("document_type", DataType::Utf8, true), + Field::new("fetch_from_source", DataType::Utf8, true), + Field::new("source_system", DataType::Utf8, true), + ] + .into(), + ), + true, + )); + + // Add the error field + fields.push(Field::new("error", DataType::Utf8, true)); + + // Create a new schema with the extended fields + Arc::new(Schema::new(fields)) +} + +fn main() -> Result<(), Box> { + // Create a Tokio runtime for running our async function + let rt = tokio::runtime::Runtime::new()?; + + // Run the function in the runtime + rt.block_on(async { test_datafusion_schema_evolution().await })?; + + println!("Example completed successfully!"); + Ok(()) +} From c1059e73671ad700860db75ab69b6cb6038fb0ec Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Tue, 10 Jun 2025 18:46:26 +0800 Subject: [PATCH 04/57] nested_struct2 <- optimizer_rule --- .../examples/nested_struct2.rs | 458 +++++++----------- 1 file changed, 174 insertions(+), 284 deletions(-) diff --git a/datafusion-examples/examples/nested_struct2.rs b/datafusion-examples/examples/nested_struct2.rs index 25ae53f6671cc..176b1a69808c1 100644 --- a/datafusion-examples/examples/nested_struct2.rs +++ b/datafusion-examples/examples/nested_struct2.rs @@ -15,317 +15,207 @@ // specific language governing permissions and limitations // under the License. -use datafusion::prelude::*; -use datafusion::{ - arrow::{ - array::Array, array::Float64Array, array::StringArray, array::StructArray, - array::TimestampMillisecondArray, datatypes::DataType, datatypes::Field, - datatypes::Schema, datatypes::TimeUnit, record_batch::RecordBatch, - }, - dataframe::DataFrameWriteOptions, - datasource::{ - file_format::parquet::ParquetFormat, - listing::ListingOptions, - listing::ListingTable, - listing::ListingTableConfig, - listing::ListingTableUrl, - // schema_adapter::SchemaAdapterFactory, - }, +use arrow::array::{ArrayRef, Int32Array, RecordBatch, StringArray}; +use arrow::datatypes::DataType; +use datafusion::common::tree_node::{Transformed, TreeNode}; +use datafusion::common::{assert_batches_eq, Result, ScalarValue}; +use datafusion::logical_expr::{ + BinaryExpr, ColumnarValue, Expr, LogicalPlan, Operator, ScalarFunctionArgs, + ScalarUDF, ScalarUDFImpl, Signature, Volatility, }; -use std::{error::Error, fs, sync::Arc}; - -/// Helper function to create a RecordBatch from a Schema and log the process -async fn create_and_write_parquet_file( - ctx: &SessionContext, - schema: &Arc, - schema_name: &str, - file_path: &str, -) -> Result<(), Box> { - let batch = create_batch(schema, schema_name)?; - - let _ = fs::remove_file(file_path); - - let df = ctx.read_batch(batch)?; - - df.write_parquet( - file_path, - DataFrameWriteOptions::default() - .with_single_file_output(true) - .with_sort_by(vec![col("timestamp_utc").sort(true, true)]), - None, - ) - .await?; +use datafusion::optimizer::ApplyOrder; +use datafusion::optimizer::{OptimizerConfig, OptimizerRule}; +use datafusion::prelude::SessionContext; +use std::any::Any; +use std::sync::Arc; + +/// This example demonstrates how to add your own [`OptimizerRule`] +/// to DataFusion. +/// +/// [`OptimizerRule`]s transform [`LogicalPlan`]s into an equivalent (but +/// hopefully faster) form. +/// +/// See [analyzer_rule.rs] for an example of AnalyzerRules, which are for +/// changing plan semantics. +#[tokio::main] +pub async fn main() -> Result<()> { + // DataFusion includes many built in OptimizerRules for tasks such as outer + // to inner join conversion and constant folding. + // + // Note you can change the order of optimizer rules using the lower level + // `SessionState` API + let ctx = SessionContext::new(); + ctx.add_optimizer_rule(Arc::new(MyOptimizerRule {})); + + // Now, let's plan and run queries with the new rule + ctx.register_batch("person", person_batch())?; + let sql = "SELECT * FROM person WHERE age = 22"; + let plan = ctx.sql(sql).await?.into_optimized_plan()?; + + // We can see the effect of our rewrite on the output plan that the filter + // has been rewritten to `my_eq` + assert_eq!( + plan.display_indent().to_string(), + "Filter: my_eq(person.age, Int32(22))\ + \n TableScan: person projection=[name, age]" + ); + + // The query below doesn't respect a filter `where age = 22` because + // the plan has been rewritten using UDF which returns always true + // + // And the output verifies the predicates have been changed (as the my_eq + // function always returns true) + assert_batches_eq!( + [ + "+--------+-----+", + "| name | age |", + "+--------+-----+", + "| Andy | 11 |", + "| Andrew | 22 |", + "| Oleks | 33 |", + "+--------+-----+", + ], + &ctx.sql(sql).await?.collect().await? + ); + + // however we can see the rule doesn't trigger for queries with predicates + // other than `=` + assert_batches_eq!( + [ + "+-------+-----+", + "| name | age |", + "+-------+-----+", + "| Andy | 11 |", + "| Oleks | 33 |", + "+-------+-----+", + ], + &ctx.sql("SELECT * FROM person WHERE age <> 22") + .await? + .collect() + .await? + ); Ok(()) } -/// Helper function to create a ListingTableConfig for given paths and schema -async fn create_listing_table_config( - ctx: &SessionContext, - paths: impl Iterator, - schema: &Arc, -) -> Result> { - let config = ListingTableConfig::new_with_multi_paths( - paths - .map(|p| ListingTableUrl::parse(&p)) - .collect::, _>>()?, - ) - .with_schema(schema.as_ref().clone().into()); +/// An example OptimizerRule that replaces all `col = ` predicates with a +/// user defined function +#[derive(Default, Debug)] +struct MyOptimizerRule {} - let config = config.infer(&ctx.state()).await?; +impl OptimizerRule for MyOptimizerRule { + fn name(&self) -> &str { + "my_optimizer_rule" + } - let updated_options = ListingOptions { - file_sort_order: vec![vec![col("timestamp_utc").sort(true, true)]], - ..config - .options - .unwrap_or_else(|| ListingOptions::new(Arc::new(ParquetFormat::default()))) - }; + // New OptimizerRules should use the "rewrite" api as it is more efficient + fn supports_rewrite(&self) -> bool { + true + } - let config = config.with_listing_options(updated_options); + /// Ask the optimizer to handle the plan recursion. `rewrite` will be called + /// on each plan node. + fn apply_order(&self) -> Option { + Some(ApplyOrder::BottomUp) + } - Ok(config) + fn rewrite( + &self, + plan: LogicalPlan, + _config: &dyn OptimizerConfig, + ) -> Result> { + plan.map_expressions(|expr| { + // This closure is called for all expressions in the current plan + // + // For example, given a plan like `SELECT a + b, 5 + 10` + // + // The closure would be called twice: + // 1. once for `a + b` + // 2. once for `5 + 10` + self.rewrite_expr(expr) + }) + } } -/// Helper function to create a ListingTable from paths and schema -async fn create_listing_table( - ctx: &SessionContext, - paths: impl Iterator, - schema: &Arc, -) -> Result, Box> { - // Create the config - let config = create_listing_table_config(ctx, paths, schema).await?; - - // Create the listing table - let listing_table = ListingTable::try_new(config)?; - - Ok(Arc::new(listing_table)) +impl MyOptimizerRule { + /// Rewrites an Expr replacing all ` = ` expressions with + /// a call to my_eq udf + fn rewrite_expr(&self, expr: Expr) -> Result> { + // do a bottom up rewrite of the expression tree + expr.transform_up(|expr| { + // Closure called for each sub tree + match expr { + Expr::BinaryExpr(binary_expr) if is_binary_eq(&binary_expr) => { + // destructure the expression + let BinaryExpr { left, op: _, right } = binary_expr; + // rewrite to `my_eq(left, right)` + let udf = ScalarUDF::new_from_impl(MyEq::new()); + let call = udf.call(vec![*left, *right]); + Ok(Transformed::yes(call)) + } + _ => Ok(Transformed::no(expr)), + } + }) + // Note that the TreeNode API handles propagating the transformed flag + // and errors up the call chain + } } -/// Helper function to register a table and execute a query -async fn execute_and_display_query( - ctx: &SessionContext, - table_name: &str, - listing_table: Arc, -) -> Result<(), Box> { - println!("==> {}", table_name); - ctx.register_table(table_name, listing_table)?; - - // Derive query automatically based on table name - let query = format!("SELECT * FROM {} ORDER BY body", table_name); - let df = ctx.sql(&query).await?; - - let _results = df.clone().collect().await?; - // Print the results - df.show().await?; - - Ok(()) +/// return true of the expression is an equality expression for a literal or +/// column reference +fn is_binary_eq(binary_expr: &BinaryExpr) -> bool { + binary_expr.op == Operator::Eq + && is_lit_or_col(binary_expr.left.as_ref()) + && is_lit_or_col(binary_expr.right.as_ref()) } -async fn test_datafusion_schema_evolution() -> Result<(), Box> { - let ctx = SessionContext::new(); - - // Create schemas - let schema1 = create_schema1(); - let schema2 = create_schema2(); - let schema3 = create_schema3(); - let schema4 = create_schema4(); - - // Define file paths in an array for easier management - let test_files = ["jobs1.parquet", "jobs2.parquet", "jobs3.parquet"]; - let [path1, path2, path3] = test_files; // Destructure for individual access - - // Create and write parquet files for each schema - create_and_write_parquet_file(&ctx, &schema1, "schema1", path1).await?; - create_and_write_parquet_file(&ctx, &schema2, "schema2", path2).await?; - create_and_write_parquet_file(&ctx, &schema3, "schema3", path3).await?; - - let paths = vec![path1.to_string(), path2.to_string(), path3.to_string()].into_iter(); - let paths2 = vec![path1.to_string(), path2.to_string(), path3.to_string()] - .into_iter() - .rev(); - - // Use the helper function to create the listing tables with different paths - let table_paths = create_listing_table(&ctx, paths, &schema4).await?; - let table_paths2 = create_listing_table(&ctx, paths2, &schema4).await?; - - // Execute query on the first table with table name "paths" - execute_and_display_query( - &ctx, - "paths", // First table with original path order - table_paths, - ) - .await?; - - // Execute query on the second table with table name "paths2" - execute_and_display_query( - &ctx, - "paths_reversed", // Second table with reversed path order - table_paths2, - ) - .await?; - - Ok(()) +/// Return true if the expression is a literal or column reference +fn is_lit_or_col(expr: &Expr) -> bool { + matches!(expr, Expr::Column(_) | Expr::Literal(_, _)) } -fn create_batch( - schema: &Arc, - schema_name: &str, -) -> Result> { - // Create arrays for each field in the schema - let columns = schema - .fields() - .iter() - .map(|field| create_array_for_field(field, 1, schema_name)) - .collect::, _>>()?; - - // Create record batch with the generated arrays - RecordBatch::try_new(schema.clone(), columns).map_err(|e| e.into()) +/// A simple user defined filter function +#[derive(Debug, Clone)] +struct MyEq { + signature: Signature, } -/// Creates an appropriate array for a given field with the specified length -fn create_array_for_field( - field: &Field, - length: usize, - schema_name: &str, -) -> Result, Box> { - match field.data_type() { - DataType::Utf8 => { - // For the body field, use schema_name; for other fields use default - if field.name() == "body" { - Ok(Arc::new(StringArray::from(vec![Some(schema_name); length]))) - } else { - let default_value = format!("{}_{}", field.name(), 1); - Ok(Arc::new(StringArray::from(vec![ - Some(default_value); - length - ]))) - } - } - DataType::Float64 => { - // Default float value - Ok(Arc::new(Float64Array::from(vec![Some(1.0); length]))) - } - DataType::Timestamp(TimeUnit::Millisecond, tz) => { - // Default timestamp (2021-12-31T12:00:00Z) - let array = - TimestampMillisecondArray::from(vec![Some(1640995200000); length]); - // Create the array with the same timezone as specified in the field - Ok(Arc::new(array.with_data_type(DataType::Timestamp( - TimeUnit::Millisecond, - tz.clone(), - )))) +impl MyEq { + fn new() -> Self { + Self { + signature: Signature::any(2, Volatility::Stable), } - DataType::Struct(fields) => { - // Create arrays for each field in the struct - let struct_arrays = fields - .iter() - .map(|f| { - let array = create_array_for_field(f, length, schema_name)?; - Ok((f.clone(), array)) - }) - .collect::, Box>>()?; - - Ok(Arc::new(StructArray::from(struct_arrays))) - } - _ => Err(format!("Unsupported data type: {}", field.data_type()).into()), } } -fn create_schema1() -> Arc { - let schema1 = Arc::new(Schema::new(vec![ - Field::new("body", DataType::Utf8, true), - Field::new( - "timestamp_utc", - DataType::Timestamp(TimeUnit::Millisecond, Some("UTC".into())), - true, - ), - ])); - schema1 -} - -/// Creates a schema with basic HTTP request fields plus a query_params struct field -fn create_schema2() -> Arc { - // Get the base schema from create_schema1 - let schema1 = create_schema1(); - - // Create a new vector of fields from schema1 - let mut fields = schema1 - .fields() - .iter() - .map(|f| f.as_ref().clone()) - .collect::>(); - - // Add the query_params field - fields.push(Field::new( - "query_params", - DataType::Struct(vec![Field::new("customer_id", DataType::Utf8, true)].into()), - true, - )); - - // Create a new schema with the extended fields - Arc::new(Schema::new(fields)) -} - -/// Creates a schema with HTTP request fields, query_params struct field, and an error field -fn create_schema3() -> Arc { - // Get the base schema from create_schema2 - let schema2 = create_schema2(); - - // Convert to a vector of fields - let mut fields = schema2 - .fields() - .iter() - .map(|f| f.as_ref().clone()) - .collect::>(); - - // Add the error field - fields.push(Field::new("error", DataType::Utf8, true)); - - // Create a new schema with the extended fields - Arc::new(Schema::new(fields)) -} - -/// Creates a schema with HTTP request fields, expanded query_params struct with additional fields, and an error field -fn create_schema4() -> Arc { - // Get the base schema from create_schema1 (we can't use schema3 directly since we need to modify query_params) - let schema1 = create_schema1(); +impl ScalarUDFImpl for MyEq { + fn as_any(&self) -> &dyn Any { + self + } - // Convert to a vector of fields - let mut fields = schema1 - .fields() - .iter() - .map(|f| f.as_ref().clone()) - .collect::>(); + fn name(&self) -> &str { + "my_eq" + } - // Add the expanded query_params field with additional fields - fields.push(Field::new( - "query_params", - DataType::Struct( - vec![ - Field::new("customer_id", DataType::Utf8, true), - Field::new("document_type", DataType::Utf8, true), - Field::new("fetch_from_source", DataType::Utf8, true), - Field::new("source_system", DataType::Utf8, true), - ] - .into(), - ), - true, - )); + fn signature(&self) -> &Signature { + &self.signature + } - // Add the error field - fields.push(Field::new("error", DataType::Utf8, true)); + fn return_type(&self, _arg_types: &[DataType]) -> Result { + Ok(DataType::Boolean) + } - // Create a new schema with the extended fields - Arc::new(Schema::new(fields)) + fn invoke_with_args(&self, _args: ScalarFunctionArgs) -> Result { + // this example simply returns "true" which is not what a real + // implementation would do. + Ok(ColumnarValue::Scalar(ScalarValue::from(true))) + } } -fn main() -> Result<(), Box> { - // Create a Tokio runtime for running our async function - let rt = tokio::runtime::Runtime::new()?; - - // Run the function in the runtime - rt.block_on(async { test_datafusion_schema_evolution().await })?; - - println!("Example completed successfully!"); - Ok(()) +/// Return a RecordBatch with made up data +fn person_batch() -> RecordBatch { + let name: ArrayRef = + Arc::new(StringArray::from_iter_values(["Andy", "Andrew", "Oleks"])); + let age: ArrayRef = Arc::new(Int32Array::from(vec![11, 22, 33])); + RecordBatch::try_from_iter(vec![("name", name), ("age", age)]).unwrap() } From 9ffeec3c81c58a9bcbb5b6d299b058e5d37074eb Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Tue, 10 Jun 2025 18:51:10 +0800 Subject: [PATCH 05/57] Revert "nested_struct2 <- optimizer_rule" This reverts commit c1059e73671ad700860db75ab69b6cb6038fb0ec. --- .../examples/nested_struct2.rs | 458 +++++++++++------- 1 file changed, 284 insertions(+), 174 deletions(-) diff --git a/datafusion-examples/examples/nested_struct2.rs b/datafusion-examples/examples/nested_struct2.rs index 176b1a69808c1..25ae53f6671cc 100644 --- a/datafusion-examples/examples/nested_struct2.rs +++ b/datafusion-examples/examples/nested_struct2.rs @@ -15,207 +15,317 @@ // specific language governing permissions and limitations // under the License. -use arrow::array::{ArrayRef, Int32Array, RecordBatch, StringArray}; -use arrow::datatypes::DataType; -use datafusion::common::tree_node::{Transformed, TreeNode}; -use datafusion::common::{assert_batches_eq, Result, ScalarValue}; -use datafusion::logical_expr::{ - BinaryExpr, ColumnarValue, Expr, LogicalPlan, Operator, ScalarFunctionArgs, - ScalarUDF, ScalarUDFImpl, Signature, Volatility, +use datafusion::prelude::*; +use datafusion::{ + arrow::{ + array::Array, array::Float64Array, array::StringArray, array::StructArray, + array::TimestampMillisecondArray, datatypes::DataType, datatypes::Field, + datatypes::Schema, datatypes::TimeUnit, record_batch::RecordBatch, + }, + dataframe::DataFrameWriteOptions, + datasource::{ + file_format::parquet::ParquetFormat, + listing::ListingOptions, + listing::ListingTable, + listing::ListingTableConfig, + listing::ListingTableUrl, + // schema_adapter::SchemaAdapterFactory, + }, }; -use datafusion::optimizer::ApplyOrder; -use datafusion::optimizer::{OptimizerConfig, OptimizerRule}; -use datafusion::prelude::SessionContext; -use std::any::Any; -use std::sync::Arc; - -/// This example demonstrates how to add your own [`OptimizerRule`] -/// to DataFusion. -/// -/// [`OptimizerRule`]s transform [`LogicalPlan`]s into an equivalent (but -/// hopefully faster) form. -/// -/// See [analyzer_rule.rs] for an example of AnalyzerRules, which are for -/// changing plan semantics. -#[tokio::main] -pub async fn main() -> Result<()> { - // DataFusion includes many built in OptimizerRules for tasks such as outer - // to inner join conversion and constant folding. - // - // Note you can change the order of optimizer rules using the lower level - // `SessionState` API - let ctx = SessionContext::new(); - ctx.add_optimizer_rule(Arc::new(MyOptimizerRule {})); - - // Now, let's plan and run queries with the new rule - ctx.register_batch("person", person_batch())?; - let sql = "SELECT * FROM person WHERE age = 22"; - let plan = ctx.sql(sql).await?.into_optimized_plan()?; - - // We can see the effect of our rewrite on the output plan that the filter - // has been rewritten to `my_eq` - assert_eq!( - plan.display_indent().to_string(), - "Filter: my_eq(person.age, Int32(22))\ - \n TableScan: person projection=[name, age]" - ); - - // The query below doesn't respect a filter `where age = 22` because - // the plan has been rewritten using UDF which returns always true - // - // And the output verifies the predicates have been changed (as the my_eq - // function always returns true) - assert_batches_eq!( - [ - "+--------+-----+", - "| name | age |", - "+--------+-----+", - "| Andy | 11 |", - "| Andrew | 22 |", - "| Oleks | 33 |", - "+--------+-----+", - ], - &ctx.sql(sql).await?.collect().await? - ); - - // however we can see the rule doesn't trigger for queries with predicates - // other than `=` - assert_batches_eq!( - [ - "+-------+-----+", - "| name | age |", - "+-------+-----+", - "| Andy | 11 |", - "| Oleks | 33 |", - "+-------+-----+", - ], - &ctx.sql("SELECT * FROM person WHERE age <> 22") - .await? - .collect() - .await? - ); +use std::{error::Error, fs, sync::Arc}; + +/// Helper function to create a RecordBatch from a Schema and log the process +async fn create_and_write_parquet_file( + ctx: &SessionContext, + schema: &Arc, + schema_name: &str, + file_path: &str, +) -> Result<(), Box> { + let batch = create_batch(schema, schema_name)?; + + let _ = fs::remove_file(file_path); + + let df = ctx.read_batch(batch)?; + + df.write_parquet( + file_path, + DataFrameWriteOptions::default() + .with_single_file_output(true) + .with_sort_by(vec![col("timestamp_utc").sort(true, true)]), + None, + ) + .await?; Ok(()) } -/// An example OptimizerRule that replaces all `col = ` predicates with a -/// user defined function -#[derive(Default, Debug)] -struct MyOptimizerRule {} +/// Helper function to create a ListingTableConfig for given paths and schema +async fn create_listing_table_config( + ctx: &SessionContext, + paths: impl Iterator, + schema: &Arc, +) -> Result> { + let config = ListingTableConfig::new_with_multi_paths( + paths + .map(|p| ListingTableUrl::parse(&p)) + .collect::, _>>()?, + ) + .with_schema(schema.as_ref().clone().into()); -impl OptimizerRule for MyOptimizerRule { - fn name(&self) -> &str { - "my_optimizer_rule" - } + let config = config.infer(&ctx.state()).await?; - // New OptimizerRules should use the "rewrite" api as it is more efficient - fn supports_rewrite(&self) -> bool { - true - } + let updated_options = ListingOptions { + file_sort_order: vec![vec![col("timestamp_utc").sort(true, true)]], + ..config + .options + .unwrap_or_else(|| ListingOptions::new(Arc::new(ParquetFormat::default()))) + }; - /// Ask the optimizer to handle the plan recursion. `rewrite` will be called - /// on each plan node. - fn apply_order(&self) -> Option { - Some(ApplyOrder::BottomUp) - } + let config = config.with_listing_options(updated_options); - fn rewrite( - &self, - plan: LogicalPlan, - _config: &dyn OptimizerConfig, - ) -> Result> { - plan.map_expressions(|expr| { - // This closure is called for all expressions in the current plan - // - // For example, given a plan like `SELECT a + b, 5 + 10` - // - // The closure would be called twice: - // 1. once for `a + b` - // 2. once for `5 + 10` - self.rewrite_expr(expr) - }) - } + Ok(config) } -impl MyOptimizerRule { - /// Rewrites an Expr replacing all ` = ` expressions with - /// a call to my_eq udf - fn rewrite_expr(&self, expr: Expr) -> Result> { - // do a bottom up rewrite of the expression tree - expr.transform_up(|expr| { - // Closure called for each sub tree - match expr { - Expr::BinaryExpr(binary_expr) if is_binary_eq(&binary_expr) => { - // destructure the expression - let BinaryExpr { left, op: _, right } = binary_expr; - // rewrite to `my_eq(left, right)` - let udf = ScalarUDF::new_from_impl(MyEq::new()); - let call = udf.call(vec![*left, *right]); - Ok(Transformed::yes(call)) - } - _ => Ok(Transformed::no(expr)), - } - }) - // Note that the TreeNode API handles propagating the transformed flag - // and errors up the call chain - } +/// Helper function to create a ListingTable from paths and schema +async fn create_listing_table( + ctx: &SessionContext, + paths: impl Iterator, + schema: &Arc, +) -> Result, Box> { + // Create the config + let config = create_listing_table_config(ctx, paths, schema).await?; + + // Create the listing table + let listing_table = ListingTable::try_new(config)?; + + Ok(Arc::new(listing_table)) } -/// return true of the expression is an equality expression for a literal or -/// column reference -fn is_binary_eq(binary_expr: &BinaryExpr) -> bool { - binary_expr.op == Operator::Eq - && is_lit_or_col(binary_expr.left.as_ref()) - && is_lit_or_col(binary_expr.right.as_ref()) +/// Helper function to register a table and execute a query +async fn execute_and_display_query( + ctx: &SessionContext, + table_name: &str, + listing_table: Arc, +) -> Result<(), Box> { + println!("==> {}", table_name); + ctx.register_table(table_name, listing_table)?; + + // Derive query automatically based on table name + let query = format!("SELECT * FROM {} ORDER BY body", table_name); + let df = ctx.sql(&query).await?; + + let _results = df.clone().collect().await?; + // Print the results + df.show().await?; + + Ok(()) } -/// Return true if the expression is a literal or column reference -fn is_lit_or_col(expr: &Expr) -> bool { - matches!(expr, Expr::Column(_) | Expr::Literal(_, _)) +async fn test_datafusion_schema_evolution() -> Result<(), Box> { + let ctx = SessionContext::new(); + + // Create schemas + let schema1 = create_schema1(); + let schema2 = create_schema2(); + let schema3 = create_schema3(); + let schema4 = create_schema4(); + + // Define file paths in an array for easier management + let test_files = ["jobs1.parquet", "jobs2.parquet", "jobs3.parquet"]; + let [path1, path2, path3] = test_files; // Destructure for individual access + + // Create and write parquet files for each schema + create_and_write_parquet_file(&ctx, &schema1, "schema1", path1).await?; + create_and_write_parquet_file(&ctx, &schema2, "schema2", path2).await?; + create_and_write_parquet_file(&ctx, &schema3, "schema3", path3).await?; + + let paths = vec![path1.to_string(), path2.to_string(), path3.to_string()].into_iter(); + let paths2 = vec![path1.to_string(), path2.to_string(), path3.to_string()] + .into_iter() + .rev(); + + // Use the helper function to create the listing tables with different paths + let table_paths = create_listing_table(&ctx, paths, &schema4).await?; + let table_paths2 = create_listing_table(&ctx, paths2, &schema4).await?; + + // Execute query on the first table with table name "paths" + execute_and_display_query( + &ctx, + "paths", // First table with original path order + table_paths, + ) + .await?; + + // Execute query on the second table with table name "paths2" + execute_and_display_query( + &ctx, + "paths_reversed", // Second table with reversed path order + table_paths2, + ) + .await?; + + Ok(()) } -/// A simple user defined filter function -#[derive(Debug, Clone)] -struct MyEq { - signature: Signature, +fn create_batch( + schema: &Arc, + schema_name: &str, +) -> Result> { + // Create arrays for each field in the schema + let columns = schema + .fields() + .iter() + .map(|field| create_array_for_field(field, 1, schema_name)) + .collect::, _>>()?; + + // Create record batch with the generated arrays + RecordBatch::try_new(schema.clone(), columns).map_err(|e| e.into()) } -impl MyEq { - fn new() -> Self { - Self { - signature: Signature::any(2, Volatility::Stable), +/// Creates an appropriate array for a given field with the specified length +fn create_array_for_field( + field: &Field, + length: usize, + schema_name: &str, +) -> Result, Box> { + match field.data_type() { + DataType::Utf8 => { + // For the body field, use schema_name; for other fields use default + if field.name() == "body" { + Ok(Arc::new(StringArray::from(vec![Some(schema_name); length]))) + } else { + let default_value = format!("{}_{}", field.name(), 1); + Ok(Arc::new(StringArray::from(vec![ + Some(default_value); + length + ]))) + } + } + DataType::Float64 => { + // Default float value + Ok(Arc::new(Float64Array::from(vec![Some(1.0); length]))) + } + DataType::Timestamp(TimeUnit::Millisecond, tz) => { + // Default timestamp (2021-12-31T12:00:00Z) + let array = + TimestampMillisecondArray::from(vec![Some(1640995200000); length]); + // Create the array with the same timezone as specified in the field + Ok(Arc::new(array.with_data_type(DataType::Timestamp( + TimeUnit::Millisecond, + tz.clone(), + )))) } + DataType::Struct(fields) => { + // Create arrays for each field in the struct + let struct_arrays = fields + .iter() + .map(|f| { + let array = create_array_for_field(f, length, schema_name)?; + Ok((f.clone(), array)) + }) + .collect::, Box>>()?; + + Ok(Arc::new(StructArray::from(struct_arrays))) + } + _ => Err(format!("Unsupported data type: {}", field.data_type()).into()), } } -impl ScalarUDFImpl for MyEq { - fn as_any(&self) -> &dyn Any { - self - } +fn create_schema1() -> Arc { + let schema1 = Arc::new(Schema::new(vec![ + Field::new("body", DataType::Utf8, true), + Field::new( + "timestamp_utc", + DataType::Timestamp(TimeUnit::Millisecond, Some("UTC".into())), + true, + ), + ])); + schema1 +} - fn name(&self) -> &str { - "my_eq" - } +/// Creates a schema with basic HTTP request fields plus a query_params struct field +fn create_schema2() -> Arc { + // Get the base schema from create_schema1 + let schema1 = create_schema1(); - fn signature(&self) -> &Signature { - &self.signature - } + // Create a new vector of fields from schema1 + let mut fields = schema1 + .fields() + .iter() + .map(|f| f.as_ref().clone()) + .collect::>(); - fn return_type(&self, _arg_types: &[DataType]) -> Result { - Ok(DataType::Boolean) - } + // Add the query_params field + fields.push(Field::new( + "query_params", + DataType::Struct(vec![Field::new("customer_id", DataType::Utf8, true)].into()), + true, + )); - fn invoke_with_args(&self, _args: ScalarFunctionArgs) -> Result { - // this example simply returns "true" which is not what a real - // implementation would do. - Ok(ColumnarValue::Scalar(ScalarValue::from(true))) - } + // Create a new schema with the extended fields + Arc::new(Schema::new(fields)) +} + +/// Creates a schema with HTTP request fields, query_params struct field, and an error field +fn create_schema3() -> Arc { + // Get the base schema from create_schema2 + let schema2 = create_schema2(); + + // Convert to a vector of fields + let mut fields = schema2 + .fields() + .iter() + .map(|f| f.as_ref().clone()) + .collect::>(); + + // Add the error field + fields.push(Field::new("error", DataType::Utf8, true)); + + // Create a new schema with the extended fields + Arc::new(Schema::new(fields)) } -/// Return a RecordBatch with made up data -fn person_batch() -> RecordBatch { - let name: ArrayRef = - Arc::new(StringArray::from_iter_values(["Andy", "Andrew", "Oleks"])); - let age: ArrayRef = Arc::new(Int32Array::from(vec![11, 22, 33])); - RecordBatch::try_from_iter(vec![("name", name), ("age", age)]).unwrap() +/// Creates a schema with HTTP request fields, expanded query_params struct with additional fields, and an error field +fn create_schema4() -> Arc { + // Get the base schema from create_schema1 (we can't use schema3 directly since we need to modify query_params) + let schema1 = create_schema1(); + + // Convert to a vector of fields + let mut fields = schema1 + .fields() + .iter() + .map(|f| f.as_ref().clone()) + .collect::>(); + + // Add the expanded query_params field with additional fields + fields.push(Field::new( + "query_params", + DataType::Struct( + vec![ + Field::new("customer_id", DataType::Utf8, true), + Field::new("document_type", DataType::Utf8, true), + Field::new("fetch_from_source", DataType::Utf8, true), + Field::new("source_system", DataType::Utf8, true), + ] + .into(), + ), + true, + )); + + // Add the error field + fields.push(Field::new("error", DataType::Utf8, true)); + + // Create a new schema with the extended fields + Arc::new(Schema::new(fields)) +} + +fn main() -> Result<(), Box> { + // Create a Tokio runtime for running our async function + let rt = tokio::runtime::Runtime::new()?; + + // Run the function in the runtime + rt.block_on(async { test_datafusion_schema_evolution().await })?; + + println!("Example completed successfully!"); + Ok(()) } From b5b3ed3e5ad183f5f8d7956018553ef6a172c2bf Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Tue, 10 Jun 2025 18:56:56 +0800 Subject: [PATCH 06/57] add nested_struct2.rs for testing --- datafusion-examples/examples/nested_struct2.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/datafusion-examples/examples/nested_struct2.rs b/datafusion-examples/examples/nested_struct2.rs index 25ae53f6671cc..b72508f5c335d 100644 --- a/datafusion-examples/examples/nested_struct2.rs +++ b/datafusion-examples/examples/nested_struct2.rs @@ -78,6 +78,7 @@ async fn create_listing_table_config( file_sort_order: vec![vec![col("timestamp_utc").sort(true, true)]], ..config .options + .clone() .unwrap_or_else(|| ListingOptions::new(Arc::new(ParquetFormat::default()))) }; From 3c711a5b1bcc30d2ec1d47551503a55c4bb0abf5 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Tue, 10 Jun 2025 19:19:13 +0800 Subject: [PATCH 07/57] feat: implement CustomParquetDataSource using NestedStructSchemaAdapterFactory --- .../examples/nested_struct2.rs | 65 +++++++++++++++++++ 1 file changed, 65 insertions(+) diff --git a/datafusion-examples/examples/nested_struct2.rs b/datafusion-examples/examples/nested_struct2.rs index b72508f5c335d..f4c9312322bfe 100644 --- a/datafusion-examples/examples/nested_struct2.rs +++ b/datafusion-examples/examples/nested_struct2.rs @@ -29,6 +29,7 @@ use datafusion::{ listing::ListingTable, listing::ListingTableConfig, listing::ListingTableUrl, + nested_schema_adapter::NestedStructSchemaAdapterFactory, // schema_adapter::SchemaAdapterFactory, }, }; @@ -59,6 +60,70 @@ async fn create_and_write_parquet_file( Ok(()) } +/// Custom DataSource that uses NestedStructSchemaAdapterFactory +use datafusion::{ + datasource::{ + source::{DataSource, DefaultTableSource}, + DataSourceExec, + }, + physical_plan::ExecutionPlan, +}; +use datafusion_physical_plan::file_scan_config::FileScanConfig; + +/// Custom DataSource implementation that configures ParquetExec with NestedStructSchemaAdapterFactory +#[derive(Debug)] +struct CustomParquetDataSource { + table_schema: Arc, + file_paths: Vec, +} + +impl CustomParquetDataSource { + fn new(table_schema: Arc, file_paths: Vec) -> Self { + Self { + table_schema, + file_paths, + } + } +} + +impl DataSource for CustomParquetDataSource { + fn create_physical_plan( + &self, + ctx: &datafusion::execution::context::TaskContext, + projection: Option<&Vec>, + filters: &[datafusion_expr::Expr], + limit: Option, + ) -> Result, datafusion_common::DataFusionError> { + // Convert file paths to PartitionedFile objects + let partitioned_files: Vec = self + .file_paths + .iter() + .map(|path| { + let metadata = std::fs::metadata(path).unwrap(); + PartitionedFile::new(path.clone(), metadata.len() as usize) + }) + .collect(); + + // Create FileScanConfig + let object_store_url = datafusion_execution::object_store::ObjectStoreUrl::local_filesystem(); + let file_scan_config = FileScanConfig::new(object_store_url, Arc::clone(&self.table_schema)) + .with_file_group(partitioned_files) + .with_projection(projection.cloned()) + .with_limit(limit); + + // Create ParquetExec with NestedStructSchemaAdapterFactory + let parquet_exec = datafusion::datasource::physical_plan::ParquetExec::builder(file_scan_config) + .with_schema_adapter_factory(Arc::new(NestedStructSchemaAdapterFactory)) + .build(); + + Ok(Arc::new(parquet_exec)) + } + + fn schema(&self) -> datafusion_common::SchemaRef { + Arc::clone(&self.table_schema) + } +} + /// Helper function to create a ListingTableConfig for given paths and schema async fn create_listing_table_config( ctx: &SessionContext, From 0e38592b030d02f4e1bb3176ffe48f996b90ba0f Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Tue, 10 Jun 2025 19:41:21 +0800 Subject: [PATCH 08/57] feat: add schema adapter factory support to ListingTableConfig --- .../core/src/datasource/listing/table.rs | 38 ++++++++++++++++++- 1 file changed, 37 insertions(+), 1 deletion(-) diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 3ddf1c85e241b..59bf0a0adf8a4 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -38,7 +38,7 @@ use datafusion_datasource::{ compute_all_files_statistics, file_groups::FileGroup, file_scan_config::{FileScanConfig, FileScanConfigBuilder}, - schema_adapter::DefaultSchemaAdapterFactory, + schema_adapter::{DefaultSchemaAdapterFactory, SchemaAdapterFactory}, }; use datafusion_execution::{ cache::{cache_manager::FileStatisticsCache, cache_unit::DefaultFileStatisticsCache}, @@ -83,6 +83,8 @@ pub struct ListingTableConfig { pub options: Option, /// Tracks the source of the schema information schema_source: SchemaSource, + /// Optional [`SchemaAdapterFactory`] for creating schema adapters + schema_adapter_factory: Option>, } impl ListingTableConfig { @@ -94,6 +96,7 @@ impl ListingTableConfig { file_schema: None, options: None, schema_source: SchemaSource::None, + schema_adapter_factory: None, } } @@ -106,6 +109,7 @@ impl ListingTableConfig { file_schema: None, options: None, schema_source: SchemaSource::None, + schema_adapter_factory: None, } } @@ -113,6 +117,7 @@ impl ListingTableConfig { pub fn schema_source(&self) -> SchemaSource { self.schema_source } + /// Set the `schema` for the overall [`ListingTable`] /// /// [`ListingTable`] will automatically coerce, when possible, the schema @@ -129,6 +134,7 @@ impl ListingTableConfig { file_schema: Some(schema), options: self.options, schema_source: SchemaSource::Specified, + schema_adapter_factory: self.schema_adapter_factory, } } @@ -142,6 +148,7 @@ impl ListingTableConfig { file_schema: self.file_schema, options: Some(listing_options), schema_source: self.schema_source, + schema_adapter_factory: self.schema_adapter_factory, } } @@ -222,6 +229,7 @@ impl ListingTableConfig { file_schema: self.file_schema, options: Some(listing_options), schema_source: self.schema_source, + schema_adapter_factory: self.schema_adapter_factory, }) } @@ -240,6 +248,7 @@ impl ListingTableConfig { file_schema, options: _, schema_source, + schema_adapter_factory, } = self; let (schema, new_schema_source) = match file_schema { @@ -261,6 +270,7 @@ impl ListingTableConfig { file_schema: Some(schema), options: Some(options), schema_source: new_schema_source, + schema_adapter_factory, }) } None => internal_err!("No `ListingOptions` set for inferring schema"), @@ -302,11 +312,37 @@ impl ListingTableConfig { file_schema: self.file_schema, options: Some(options), schema_source: self.schema_source, + schema_adapter_factory: self.schema_adapter_factory, }) } None => config_err!("No `ListingOptions` set for inferring schema"), } } + + /// Set the [`SchemaAdapterFactory`] for the [`ListingTable`] + /// + /// The schema adapter factory is used to create schema adapters that can + /// handle schema evolution and type conversions when reading files with + /// different schemas than the table schema. + /// + /// If not provided, a default schema adapter factory will be used. + pub fn with_schema_adapter_factory( + self, + schema_adapter_factory: Arc, + ) -> Self { + Self { + table_paths: self.table_paths, + file_schema: self.file_schema, + options: self.options, + schema_source: self.schema_source, + schema_adapter_factory: Some(schema_adapter_factory), + } + } + + /// Get the [`SchemaAdapterFactory`] for this configuration + pub fn schema_adapter_factory(&self) -> Option<&Arc> { + self.schema_adapter_factory.as_ref() + } } /// Options for creating a [`ListingTable`] From e8f5b95677bbfde6d83bc45ce31b4102097d3a88 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Tue, 10 Jun 2025 20:01:00 +0800 Subject: [PATCH 09/57] feat: ListingTable apply schema adapter to file source --- .../core/src/datasource/listing/table.rs | 29 ++++++++++++++++++- 1 file changed, 28 insertions(+), 1 deletion(-) diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 59bf0a0adf8a4..39c3c9835a5db 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -837,6 +837,8 @@ pub struct ListingTable { collected_statistics: FileStatisticsCache, constraints: Constraints, column_defaults: HashMap, + /// Optional [`SchemaAdapterFactory`] for creating schema adapters + schema_adapter_factory: Option>, } impl ListingTable { @@ -877,6 +879,7 @@ impl ListingTable { collected_statistics: Arc::new(DefaultFileStatisticsCache::default()), constraints: Constraints::default(), column_defaults: HashMap::new(), + schema_adapter_factory: config.schema_adapter_factory, }; Ok(table) @@ -930,6 +933,24 @@ impl ListingTable { self.schema_source } + /// Set the [`SchemaAdapterFactory`] for this [`ListingTable`] + /// + /// The schema adapter factory is used to create schema adapters that can + /// handle schema evolution and type conversions when reading files with + /// different schemas than 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 + } + + /// Get the [`SchemaAdapterFactory`] for this table + pub fn schema_adapter_factory(&self) -> Option<&Arc> { + self.schema_adapter_factory.as_ref() + } + /// If file_sort_order is specified, creates the appropriate physical expressions fn try_create_output_ordering(&self) -> Result> { create_ordering(&self.table_schema, &self.options.file_sort_order) @@ -1038,6 +1059,12 @@ impl TableProvider for ListingTable { return Ok(Arc::new(EmptyExec::new(Arc::new(Schema::empty())))); }; + let mut source = self.options.format.file_source(); + // Apply schema adapter to source if available + if let Some(factory) = &self.schema_adapter_factory { + source = source.with_schema_adapter_factory(Arc::clone(factory))?; + } + // create the execution plan self.options .format @@ -1046,7 +1073,7 @@ impl TableProvider for ListingTable { FileScanConfigBuilder::new( object_store_url, Arc::clone(&self.file_schema), - self.options.format.file_source(), + source, ) .with_file_groups(partitioned_file_lists) .with_constraints(self.constraints.clone()) From 6fcf1a7f9201924c6a3c322f1855f4b74f6217e5 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Tue, 10 Jun 2025 20:08:50 +0800 Subject: [PATCH 10/57] feat: nested_struct2 - fix test --- .../examples/nested_struct2.rs | 74 ++----------------- 1 file changed, 5 insertions(+), 69 deletions(-) diff --git a/datafusion-examples/examples/nested_struct2.rs b/datafusion-examples/examples/nested_struct2.rs index f4c9312322bfe..bdd3073bd6ac4 100644 --- a/datafusion-examples/examples/nested_struct2.rs +++ b/datafusion-examples/examples/nested_struct2.rs @@ -25,12 +25,8 @@ use datafusion::{ dataframe::DataFrameWriteOptions, datasource::{ file_format::parquet::ParquetFormat, - listing::ListingOptions, - listing::ListingTable, - listing::ListingTableConfig, - listing::ListingTableUrl, + listing::{ListingOptions, ListingTable, ListingTableConfig, ListingTableUrl}, nested_schema_adapter::NestedStructSchemaAdapterFactory, - // schema_adapter::SchemaAdapterFactory, }, }; use std::{error::Error, fs, sync::Arc}; @@ -60,70 +56,6 @@ async fn create_and_write_parquet_file( Ok(()) } -/// Custom DataSource that uses NestedStructSchemaAdapterFactory -use datafusion::{ - datasource::{ - source::{DataSource, DefaultTableSource}, - DataSourceExec, - }, - physical_plan::ExecutionPlan, -}; -use datafusion_physical_plan::file_scan_config::FileScanConfig; - -/// Custom DataSource implementation that configures ParquetExec with NestedStructSchemaAdapterFactory -#[derive(Debug)] -struct CustomParquetDataSource { - table_schema: Arc, - file_paths: Vec, -} - -impl CustomParquetDataSource { - fn new(table_schema: Arc, file_paths: Vec) -> Self { - Self { - table_schema, - file_paths, - } - } -} - -impl DataSource for CustomParquetDataSource { - fn create_physical_plan( - &self, - ctx: &datafusion::execution::context::TaskContext, - projection: Option<&Vec>, - filters: &[datafusion_expr::Expr], - limit: Option, - ) -> Result, datafusion_common::DataFusionError> { - // Convert file paths to PartitionedFile objects - let partitioned_files: Vec = self - .file_paths - .iter() - .map(|path| { - let metadata = std::fs::metadata(path).unwrap(); - PartitionedFile::new(path.clone(), metadata.len() as usize) - }) - .collect(); - - // Create FileScanConfig - let object_store_url = datafusion_execution::object_store::ObjectStoreUrl::local_filesystem(); - let file_scan_config = FileScanConfig::new(object_store_url, Arc::clone(&self.table_schema)) - .with_file_group(partitioned_files) - .with_projection(projection.cloned()) - .with_limit(limit); - - // Create ParquetExec with NestedStructSchemaAdapterFactory - let parquet_exec = datafusion::datasource::physical_plan::ParquetExec::builder(file_scan_config) - .with_schema_adapter_factory(Arc::new(NestedStructSchemaAdapterFactory)) - .build(); - - Ok(Arc::new(parquet_exec)) - } - - fn schema(&self) -> datafusion_common::SchemaRef { - Arc::clone(&self.table_schema) - } -} - /// Helper function to create a ListingTableConfig for given paths and schema async fn create_listing_table_config( ctx: &SessionContext, @@ -161,6 +93,10 @@ async fn create_listing_table( // Create the config let config = create_listing_table_config(ctx, paths, schema).await?; + // Add the NestedStructSchemaAdapterFactory to handle schema evolution + let config = + config.with_schema_adapter_factory(Arc::new(NestedStructSchemaAdapterFactory)); + // Create the listing table let listing_table = ListingTable::try_new(config)?; From f1701698944a86700b6def26251ac5696acf216a Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Tue, 10 Jun 2025 22:50:19 +0800 Subject: [PATCH 11/57] feat: remove nested_struct2 example file --- .../examples/nested_struct2.rs | 333 ------------------ 1 file changed, 333 deletions(-) delete mode 100644 datafusion-examples/examples/nested_struct2.rs diff --git a/datafusion-examples/examples/nested_struct2.rs b/datafusion-examples/examples/nested_struct2.rs deleted file mode 100644 index bdd3073bd6ac4..0000000000000 --- a/datafusion-examples/examples/nested_struct2.rs +++ /dev/null @@ -1,333 +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 datafusion::prelude::*; -use datafusion::{ - arrow::{ - array::Array, array::Float64Array, array::StringArray, array::StructArray, - array::TimestampMillisecondArray, datatypes::DataType, datatypes::Field, - datatypes::Schema, datatypes::TimeUnit, record_batch::RecordBatch, - }, - dataframe::DataFrameWriteOptions, - datasource::{ - file_format::parquet::ParquetFormat, - listing::{ListingOptions, ListingTable, ListingTableConfig, ListingTableUrl}, - nested_schema_adapter::NestedStructSchemaAdapterFactory, - }, -}; -use std::{error::Error, fs, sync::Arc}; - -/// Helper function to create a RecordBatch from a Schema and log the process -async fn create_and_write_parquet_file( - ctx: &SessionContext, - schema: &Arc, - schema_name: &str, - file_path: &str, -) -> Result<(), Box> { - let batch = create_batch(schema, schema_name)?; - - let _ = fs::remove_file(file_path); - - let df = ctx.read_batch(batch)?; - - df.write_parquet( - file_path, - DataFrameWriteOptions::default() - .with_single_file_output(true) - .with_sort_by(vec![col("timestamp_utc").sort(true, true)]), - None, - ) - .await?; - - Ok(()) -} - -/// Helper function to create a ListingTableConfig for given paths and schema -async fn create_listing_table_config( - ctx: &SessionContext, - paths: impl Iterator, - schema: &Arc, -) -> Result> { - let config = ListingTableConfig::new_with_multi_paths( - paths - .map(|p| ListingTableUrl::parse(&p)) - .collect::, _>>()?, - ) - .with_schema(schema.as_ref().clone().into()); - - let config = config.infer(&ctx.state()).await?; - - let updated_options = ListingOptions { - file_sort_order: vec![vec![col("timestamp_utc").sort(true, true)]], - ..config - .options - .clone() - .unwrap_or_else(|| ListingOptions::new(Arc::new(ParquetFormat::default()))) - }; - - let config = config.with_listing_options(updated_options); - - Ok(config) -} - -/// Helper function to create a ListingTable from paths and schema -async fn create_listing_table( - ctx: &SessionContext, - paths: impl Iterator, - schema: &Arc, -) -> Result, Box> { - // Create the config - let config = create_listing_table_config(ctx, paths, schema).await?; - - // Add the NestedStructSchemaAdapterFactory to handle schema evolution - let config = - config.with_schema_adapter_factory(Arc::new(NestedStructSchemaAdapterFactory)); - - // Create the listing table - let listing_table = ListingTable::try_new(config)?; - - Ok(Arc::new(listing_table)) -} - -/// Helper function to register a table and execute a query -async fn execute_and_display_query( - ctx: &SessionContext, - table_name: &str, - listing_table: Arc, -) -> Result<(), Box> { - println!("==> {}", table_name); - ctx.register_table(table_name, listing_table)?; - - // Derive query automatically based on table name - let query = format!("SELECT * FROM {} ORDER BY body", table_name); - let df = ctx.sql(&query).await?; - - let _results = df.clone().collect().await?; - // Print the results - df.show().await?; - - Ok(()) -} - -async fn test_datafusion_schema_evolution() -> Result<(), Box> { - let ctx = SessionContext::new(); - - // Create schemas - let schema1 = create_schema1(); - let schema2 = create_schema2(); - let schema3 = create_schema3(); - let schema4 = create_schema4(); - - // Define file paths in an array for easier management - let test_files = ["jobs1.parquet", "jobs2.parquet", "jobs3.parquet"]; - let [path1, path2, path3] = test_files; // Destructure for individual access - - // Create and write parquet files for each schema - create_and_write_parquet_file(&ctx, &schema1, "schema1", path1).await?; - create_and_write_parquet_file(&ctx, &schema2, "schema2", path2).await?; - create_and_write_parquet_file(&ctx, &schema3, "schema3", path3).await?; - - let paths = vec![path1.to_string(), path2.to_string(), path3.to_string()].into_iter(); - let paths2 = vec![path1.to_string(), path2.to_string(), path3.to_string()] - .into_iter() - .rev(); - - // Use the helper function to create the listing tables with different paths - let table_paths = create_listing_table(&ctx, paths, &schema4).await?; - let table_paths2 = create_listing_table(&ctx, paths2, &schema4).await?; - - // Execute query on the first table with table name "paths" - execute_and_display_query( - &ctx, - "paths", // First table with original path order - table_paths, - ) - .await?; - - // Execute query on the second table with table name "paths2" - execute_and_display_query( - &ctx, - "paths_reversed", // Second table with reversed path order - table_paths2, - ) - .await?; - - Ok(()) -} - -fn create_batch( - schema: &Arc, - schema_name: &str, -) -> Result> { - // Create arrays for each field in the schema - let columns = schema - .fields() - .iter() - .map(|field| create_array_for_field(field, 1, schema_name)) - .collect::, _>>()?; - - // Create record batch with the generated arrays - RecordBatch::try_new(schema.clone(), columns).map_err(|e| e.into()) -} - -/// Creates an appropriate array for a given field with the specified length -fn create_array_for_field( - field: &Field, - length: usize, - schema_name: &str, -) -> Result, Box> { - match field.data_type() { - DataType::Utf8 => { - // For the body field, use schema_name; for other fields use default - if field.name() == "body" { - Ok(Arc::new(StringArray::from(vec![Some(schema_name); length]))) - } else { - let default_value = format!("{}_{}", field.name(), 1); - Ok(Arc::new(StringArray::from(vec![ - Some(default_value); - length - ]))) - } - } - DataType::Float64 => { - // Default float value - Ok(Arc::new(Float64Array::from(vec![Some(1.0); length]))) - } - DataType::Timestamp(TimeUnit::Millisecond, tz) => { - // Default timestamp (2021-12-31T12:00:00Z) - let array = - TimestampMillisecondArray::from(vec![Some(1640995200000); length]); - // Create the array with the same timezone as specified in the field - Ok(Arc::new(array.with_data_type(DataType::Timestamp( - TimeUnit::Millisecond, - tz.clone(), - )))) - } - DataType::Struct(fields) => { - // Create arrays for each field in the struct - let struct_arrays = fields - .iter() - .map(|f| { - let array = create_array_for_field(f, length, schema_name)?; - Ok((f.clone(), array)) - }) - .collect::, Box>>()?; - - Ok(Arc::new(StructArray::from(struct_arrays))) - } - _ => Err(format!("Unsupported data type: {}", field.data_type()).into()), - } -} - -fn create_schema1() -> Arc { - let schema1 = Arc::new(Schema::new(vec![ - Field::new("body", DataType::Utf8, true), - Field::new( - "timestamp_utc", - DataType::Timestamp(TimeUnit::Millisecond, Some("UTC".into())), - true, - ), - ])); - schema1 -} - -/// Creates a schema with basic HTTP request fields plus a query_params struct field -fn create_schema2() -> Arc { - // Get the base schema from create_schema1 - let schema1 = create_schema1(); - - // Create a new vector of fields from schema1 - let mut fields = schema1 - .fields() - .iter() - .map(|f| f.as_ref().clone()) - .collect::>(); - - // Add the query_params field - fields.push(Field::new( - "query_params", - DataType::Struct(vec![Field::new("customer_id", DataType::Utf8, true)].into()), - true, - )); - - // Create a new schema with the extended fields - Arc::new(Schema::new(fields)) -} - -/// Creates a schema with HTTP request fields, query_params struct field, and an error field -fn create_schema3() -> Arc { - // Get the base schema from create_schema2 - let schema2 = create_schema2(); - - // Convert to a vector of fields - let mut fields = schema2 - .fields() - .iter() - .map(|f| f.as_ref().clone()) - .collect::>(); - - // Add the error field - fields.push(Field::new("error", DataType::Utf8, true)); - - // Create a new schema with the extended fields - Arc::new(Schema::new(fields)) -} - -/// Creates a schema with HTTP request fields, expanded query_params struct with additional fields, and an error field -fn create_schema4() -> Arc { - // Get the base schema from create_schema1 (we can't use schema3 directly since we need to modify query_params) - let schema1 = create_schema1(); - - // Convert to a vector of fields - let mut fields = schema1 - .fields() - .iter() - .map(|f| f.as_ref().clone()) - .collect::>(); - - // Add the expanded query_params field with additional fields - fields.push(Field::new( - "query_params", - DataType::Struct( - vec![ - Field::new("customer_id", DataType::Utf8, true), - Field::new("document_type", DataType::Utf8, true), - Field::new("fetch_from_source", DataType::Utf8, true), - Field::new("source_system", DataType::Utf8, true), - ] - .into(), - ), - true, - )); - - // Add the error field - fields.push(Field::new("error", DataType::Utf8, true)); - - // Create a new schema with the extended fields - Arc::new(Schema::new(fields)) -} - -fn main() -> Result<(), Box> { - // Create a Tokio runtime for running our async function - let rt = tokio::runtime::Runtime::new()?; - - // Run the function in the runtime - rt.block_on(async { test_datafusion_schema_evolution().await })?; - - println!("Example completed successfully!"); - Ok(()) -} From f6997cd82528d32685e663c095c6ec20c862c9ee Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Tue, 10 Jun 2025 22:59:49 +0800 Subject: [PATCH 12/57] refactor: reorganize imports in nested_schema_adapter.rs for clarity --- .../datasource/src/nested_schema_adapter.rs | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/datafusion/datasource/src/nested_schema_adapter.rs b/datafusion/datasource/src/nested_schema_adapter.rs index 4af80b1c061e1..5d6369a0b9275 100644 --- a/datafusion/datasource/src/nested_schema_adapter.rs +++ b/datafusion/datasource/src/nested_schema_adapter.rs @@ -21,18 +21,18 @@ //! physical format into how they should be used by DataFusion. For instance, a schema //! can be stored external to a parquet file that maps parquet logical types to arrow types. -use arrow::datatypes::{DataType::Struct, Field, Schema, SchemaRef}; -use datafusion_common::{ColumnStatistics, Result}; -use std::sync::Arc; - use crate::schema_adapter::{ create_field_mapping, DefaultSchemaAdapterFactory, SchemaAdapter, SchemaAdapterFactory, SchemaMapper, }; -use arrow::array::{Array, ArrayRef, StructArray}; -use arrow::compute::cast; -use arrow::record_batch::{RecordBatch, RecordBatchOptions}; -use datafusion_common::arrow::array::new_null_array; +use arrow::{ + array::{Array, ArrayRef, StructArray}, + compute::cast, + datatypes::{DataType::Struct, Field, Schema, SchemaRef}, + record_batch::{RecordBatch, RecordBatchOptions}, +}; +use datafusion_common::{arrow::array::new_null_array, ColumnStatistics, Result}; +use std::sync::Arc; /// Factory for creating [`NestedStructSchemaAdapter`] /// @@ -312,10 +312,10 @@ fn adapt_column(source_col: &ArrayRef, target_field: &Field) -> Result #[cfg(test)] mod tests { use super::*; - use arrow::array::{Array, StringBuilder, StructArray, TimestampMillisecondArray}; - use arrow::datatypes::{ - DataType::{Float64, Int16, Int32, Timestamp, Utf8}, - TimeUnit::Millisecond, + use arrow::{ + array::{Array, StringBuilder, StructArray, TimestampMillisecondArray}, + datatypes::DataType::{Float64, Int16, Int32, Timestamp, Utf8}, + datatypes::TimeUnit::Millisecond, }; use datafusion_common::ScalarValue; From 3fb1b70e5a2fb4eb1dda7c75d16488c0b60e370d Mon Sep 17 00:00:00 2001 From: kosiew Date: Wed, 11 Jun 2025 17:36:03 +0800 Subject: [PATCH 13/57] feat: use custom schema adapter factory Summary use the table's SchemaAdapterFactory in list_files_for_scan add a test for statistics mapping via a custom factory Testing cargo test -p datafusion --lib test_statistics_mapping_with_custom_factory -- --nocapture cargo test -p datafusion --lib --quiet (fails: missing test data) --- .../core/src/datasource/listing/table.rs | 103 +++++++++++++++++- 1 file changed, 100 insertions(+), 3 deletions(-) diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 39c3c9835a5db..e73dfdffedf92 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -38,7 +38,7 @@ use datafusion_datasource::{ compute_all_files_statistics, file_groups::FileGroup, file_scan_config::{FileScanConfig, FileScanConfigBuilder}, - schema_adapter::{DefaultSchemaAdapterFactory, SchemaAdapterFactory}, + schema_adapter::{DefaultSchemaAdapterFactory, SchemaAdapter, SchemaAdapterFactory}, }; use datafusion_execution::{ cache::{cache_manager::FileStatisticsCache, cache_unit::DefaultFileStatisticsCache}, @@ -1232,8 +1232,14 @@ impl ListingTable { self.options.collect_stat, inexact_stats, )?; - let (schema_mapper, _) = DefaultSchemaAdapterFactory::from_schema(self.schema()) - .map_schema(self.file_schema.as_ref())?; + let table_schema = self.schema(); + let schema_adapter: Box = match &self.schema_adapter_factory { + Some(factory) => { + factory.create(Arc::clone(&table_schema), Arc::clone(&table_schema)) + } + None => DefaultSchemaAdapterFactory::from_schema(Arc::clone(&table_schema)), + }; + let (schema_mapper, _) = schema_adapter.map_schema(self.file_schema.as_ref())?; stats.column_statistics = schema_mapper.map_column_statistics(&stats.column_statistics)?; file_groups.iter_mut().try_for_each(|file_group| { @@ -1379,12 +1385,16 @@ mod tests { }, }; use arrow::{compute::SortOptions, record_batch::RecordBatch}; + use datafusion_common::ColumnStatistics; use datafusion_common::{ assert_contains, stats::Precision, test_util::{batches_to_string, datafusion_test_data}, ScalarValue, }; + use datafusion_datasource::schema_adapter::{ + SchemaAdapter, SchemaAdapterFactory, SchemaMapper, + }; use datafusion_expr::{BinaryExpr, LogicalPlanBuilder, Operator}; use datafusion_physical_expr::PhysicalSortExpr; use datafusion_physical_plan::{collect, ExecutionPlanProperties}; @@ -2616,4 +2626,91 @@ mod tests { Ok(()) } + + #[tokio::test] + async fn test_statistics_mapping_with_custom_factory() -> Result<()> { + let ctx = SessionContext::new(); + let path = "table/file.json"; + register_test_store(&ctx, &[(path, 10)]); + + let format = JsonFormat::default(); + let opt = ListingOptions::new(Arc::new(format)).with_collect_stat(false); + let schema = Schema::new(vec![Field::new("a", DataType::Boolean, false)]); + let table_path = ListingTableUrl::parse("test:///table/").unwrap(); + + let config = ListingTableConfig::new(table_path) + .with_listing_options(opt) + .with_schema(Arc::new(schema)) + .with_schema_adapter_factory(Arc::new(NullStatsAdapterFactory {})); + let table = ListingTable::try_new(config)?; + + let (groups, stats) = table.list_files_for_scan(&ctx.state(), &[], None).await?; + + assert_eq!(stats.column_statistics[0].null_count, Precision::Exact(42)); + for g in groups { + if let Some(s) = g.file_statistics(None) { + assert_eq!(s.column_statistics[0].null_count, Precision::Exact(42)); + } + } + + Ok(()) + } + + #[derive(Debug)] + struct NullStatsAdapterFactory; + + impl SchemaAdapterFactory for NullStatsAdapterFactory { + fn create( + &self, + projected_table_schema: SchemaRef, + _table_schema: SchemaRef, + ) -> Box { + Box::new(NullStatsAdapter { + schema: projected_table_schema, + }) + } + } + + #[derive(Debug)] + struct NullStatsAdapter { + schema: SchemaRef, + } + + impl SchemaAdapter for NullStatsAdapter { + fn map_column_index(&self, index: usize, file_schema: &Schema) -> Option { + let field = self.schema.field(index); + file_schema.fields.find(field.name()).map(|(i, _)| i) + } + + fn map_schema( + &self, + file_schema: &Schema, + ) -> Result<(Arc, Vec)> { + let projection = (0..file_schema.fields().len()).collect(); + Ok((Arc::new(NullStatsMapper {}), projection)) + } + } + + #[derive(Debug)] + struct NullStatsMapper; + + impl SchemaMapper for NullStatsMapper { + fn map_batch(&self, batch: RecordBatch) -> Result { + Ok(batch) + } + + fn map_column_statistics( + &self, + stats: &[ColumnStatistics], + ) -> Result> { + Ok(stats + .iter() + .map(|s| { + let mut s = s.clone(); + s.null_count = Precision::Exact(42); + s + }) + .collect()) + } + } } From 11b74630dde006cc8ab314ba4eb155e10efd28d9 Mon Sep 17 00:00:00 2001 From: kosiew Date: Wed, 11 Jun 2025 17:37:47 +0800 Subject: [PATCH 14/57] refactor schema mapping with column adapter allow SchemaMapping to use a custom per-column adaptation function update DefaultSchemaAdapter and NestedStructSchemaAdapter to pass closures remove NestedStructSchemaMapping adjust tests --- .../datasource/src/nested_schema_adapter.rs | 112 +----------------- datafusion/datasource/src/schema_adapter.rs | 42 +++++-- 2 files changed, 39 insertions(+), 115 deletions(-) diff --git a/datafusion/datasource/src/nested_schema_adapter.rs b/datafusion/datasource/src/nested_schema_adapter.rs index 5d6369a0b9275..86585ac602ba2 100644 --- a/datafusion/datasource/src/nested_schema_adapter.rs +++ b/datafusion/datasource/src/nested_schema_adapter.rs @@ -23,13 +23,13 @@ use crate::schema_adapter::{ create_field_mapping, DefaultSchemaAdapterFactory, SchemaAdapter, - SchemaAdapterFactory, SchemaMapper, + SchemaAdapterFactory, SchemaMapper, SchemaMapping, }; use arrow::{ array::{Array, ArrayRef, StructArray}, compute::cast, datatypes::{DataType::Struct, Field, Schema, SchemaRef}, - record_batch::{RecordBatch, RecordBatchOptions}, + record_batch::RecordBatch, }; use datafusion_common::{arrow::array::new_null_array, ColumnStatistics, Result}; use std::sync::Arc; @@ -149,119 +149,17 @@ impl SchemaAdapter for NestedStructSchemaAdapter { )?; Ok(( - Arc::new(NestedStructSchemaMapping::new( + Arc::new(SchemaMapping::new( Arc::clone(&self.projected_table_schema), field_mappings, + Arc::new(|array: &ArrayRef, field: &Field| Ok(adapt_column(array, field)?)), )), projection, )) } } -/// A SchemaMapping implementation specifically for nested structs -#[derive(Debug)] -struct NestedStructSchemaMapping { - /// The schema for the table, projected to include only the fields being output - projected_table_schema: SchemaRef, - /// Field mappings from projected table to file schema - field_mappings: Vec>, -} - -impl NestedStructSchemaMapping { - /// Create a new nested struct schema mapping - pub fn new( - projected_table_schema: SchemaRef, - field_mappings: Vec>, - ) -> Self { - Self { - projected_table_schema, - field_mappings, - } - } -} - -/// Maps a `RecordBatch` to a new `RecordBatch` according to the schema mapping defined in `NestedStructSchemaMapping`. -/// -/// # Arguments -/// -/// * `batch` - The input `RecordBatch` to be mapped. -/// -/// # Returns -/// -/// A `Result` containing the new `RecordBatch` with columns adapted according to the schema mapping, or an error if the mapping fails. -/// -/// # Behavior -/// -/// - For each field in the projected table schema, the corresponding column in the input batch is adapted. -/// - If a field does not exist in the input batch, a null array of the appropriate data type and length is created and used in the output batch. -/// - If a field exists in the input batch, the column is adapted to handle potential nested struct adaptation. -/// -/// # Errors -/// -/// Returns an error if the column adaptation fails or if the new `RecordBatch` cannot be created. -impl SchemaMapper for NestedStructSchemaMapping { - fn map_batch(&self, batch: RecordBatch) -> Result { - let batch_rows = batch.num_rows(); - let batch_cols = batch.columns().to_vec(); - - let cols = self - .projected_table_schema - .fields() - .iter() - .zip(&self.field_mappings) - .map(|(field, file_idx)| { - file_idx.map_or_else( - // If field doesn't exist in file, return null array - || Ok(new_null_array(field.data_type(), batch_rows)), - // If field exists, handle potential nested struct adaptation - |batch_idx| adapt_column(&batch_cols[batch_idx], field), - ) - }) - .collect::, _>>()?; - - // Create record batch with adapted columns - let options = RecordBatchOptions::new().with_row_count(Some(batch.num_rows())); - let schema = Arc::clone(&self.projected_table_schema); - let record_batch = RecordBatch::try_new_with_options(schema, cols, &options)?; - Ok(record_batch) - } - - /// Adapts file-level column `Statistics` to match the `table_schema` - /// - /// Maps statistics from the file schema to the projected table schema using field mappings. - /// For fields not present in the file schema, uses unknown statistics. - fn map_column_statistics( - &self, - file_col_statistics: &[ColumnStatistics], - ) -> Result> { - let mut table_col_statistics = vec![]; - - // Map statistics for each field based on field_mappings - for (_, file_col_idx) in self - .projected_table_schema - .fields() - .iter() - .zip(&self.field_mappings) - { - if let Some(file_col_idx) = file_col_idx { - // Use statistics from file if available, otherwise default - table_col_statistics.push( - file_col_statistics - .get(*file_col_idx) - .cloned() - .unwrap_or_default(), - ); - } else { - // Field doesn't exist in file schema, use unknown statistics - table_col_statistics.push(ColumnStatistics::new_unknown()); - } - } - - Ok(table_col_statistics) - } -} - -// Helper methods for the NestedStructSchemaMapping +// Helper methods for nested struct adaptation /// Adapt a column to match the target field type, handling nested structs specially fn adapt_column(source_col: &ArrayRef, target_field: &Field) -> Result { match target_field.data_type() { diff --git a/datafusion/datasource/src/schema_adapter.rs b/datafusion/datasource/src/schema_adapter.rs index 519be97a81021..6dbba2110c254 100644 --- a/datafusion/datasource/src/schema_adapter.rs +++ b/datafusion/datasource/src/schema_adapter.rs @@ -21,13 +21,18 @@ //! physical format into how they should be used by DataFusion. For instance, a schema //! can be stored external to a parquet file that maps parquet logical types to arrow types. -use arrow::array::{new_null_array, RecordBatch, RecordBatchOptions}; +use arrow::array::{new_null_array, ArrayRef, RecordBatch, RecordBatchOptions}; use arrow::compute::{can_cast_types, cast}; use arrow::datatypes::{Field, Schema, SchemaRef}; use datafusion_common::{plan_err, ColumnStatistics}; use std::fmt::Debug; use std::sync::Arc; +/// Function used by [`SchemaMapping`] to adapt a column from the file schema to +/// the table schema. +pub type AdaptColumnFn = + dyn Fn(&ArrayRef, &Field) -> datafusion_common::Result + Send + Sync; + /// Factory for creating [`SchemaAdapter`] /// /// This interface provides a way to implement custom schema adaptation logic @@ -277,6 +282,7 @@ impl SchemaAdapter for DefaultSchemaAdapter { Arc::new(SchemaMapping::new( Arc::clone(&self.projected_table_schema), field_mappings, + Arc::new(|array: &ArrayRef, field: &Field| Ok(cast(array, field.data_type())?)), )), projection, )) @@ -323,7 +329,6 @@ where /// `projected_table_schema` as it can only operate on the projected fields. /// /// [`map_batch`]: Self::map_batch -#[derive(Debug)] pub struct SchemaMapping { /// The schema of the table. This is the expected schema after conversion /// and it should match the schema of the query result. @@ -334,6 +339,18 @@ pub struct SchemaMapping { /// They are Options instead of just plain `usize`s because the table could /// have fields that don't exist in the file. field_mappings: Vec>, + /// Function used to adapt a column from the file schema to the table schema + /// when it exists in both schemas + adapt_column: Arc, +} + +impl Debug for SchemaMapping { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + f.debug_struct("SchemaMapping") + .field("projected_table_schema", &self.projected_table_schema) + .field("field_mappings", &self.field_mappings) + .finish() + } } impl SchemaMapping { @@ -343,10 +360,12 @@ impl SchemaMapping { pub fn new( projected_table_schema: SchemaRef, field_mappings: Vec>, + adapt_column: Arc, ) -> Self { Self { projected_table_schema, field_mappings, + adapt_column, } } } @@ -373,9 +392,9 @@ impl SchemaMapper for SchemaMapping { // If this field only exists in the table, and not in the file, then we know // that it's null, so just return that. || Ok(new_null_array(field.data_type(), batch_rows)), - // However, if it does exist in both, then try to cast it to the correct output - // type - |batch_idx| cast(&batch_cols[batch_idx], field.data_type()), + // However, if it does exist in both, use the adapt_column function + // to perform any necessary conversions + |batch_idx| (self.adapt_column)(&batch_cols[batch_idx], field), ) }) .collect::, _>>()?; @@ -421,7 +440,11 @@ impl SchemaMapper for SchemaMapping { #[cfg(test)] mod tests { - use arrow::datatypes::{DataType, Field}; + use arrow::{ + array::ArrayRef, + compute::cast, + datatypes::{DataType, Field}, + }; use datafusion_common::{stats::Precision, Statistics}; use super::*; @@ -595,8 +618,11 @@ mod tests { let field_mappings = vec![Some(1), Some(0)]; // Create SchemaMapping manually - let mapping = - SchemaMapping::new(Arc::clone(&projected_schema), field_mappings.clone()); + let mapping = SchemaMapping::new( + Arc::clone(&projected_schema), + field_mappings.clone(), + Arc::new(|array: &ArrayRef, field: &Field| Ok(cast(array, field.data_type())?)), + ); // Check that fields were set correctly assert_eq!(*mapping.projected_table_schema, *projected_schema); From 368774f7dd2bfd43c0738ad1fd49d41b4066466a Mon Sep 17 00:00:00 2001 From: kosiew Date: Wed, 11 Jun 2025 17:38:10 +0800 Subject: [PATCH 15/57] Remove unused create_adapter --- .../datasource/src/nested_schema_adapter.rs | 33 +++++++------------ 1 file changed, 12 insertions(+), 21 deletions(-) diff --git a/datafusion/datasource/src/nested_schema_adapter.rs b/datafusion/datasource/src/nested_schema_adapter.rs index 5d6369a0b9275..2f9af6cf824c1 100644 --- a/datafusion/datasource/src/nested_schema_adapter.rs +++ b/datafusion/datasource/src/nested_schema_adapter.rs @@ -22,9 +22,10 @@ //! can be stored external to a parquet file that maps parquet logical types to arrow types. use crate::schema_adapter::{ - create_field_mapping, DefaultSchemaAdapterFactory, SchemaAdapter, - SchemaAdapterFactory, SchemaMapper, + create_field_mapping, SchemaAdapter, SchemaAdapterFactory, SchemaMapper, }; +#[cfg(test)] +use crate::schema_adapter::DefaultSchemaAdapterFactory; use arrow::{ array::{Array, ArrayRef, StructArray}, compute::cast, @@ -73,21 +74,6 @@ impl NestedStructSchemaAdapterFactory { .any(|field| matches!(field.data_type(), Struct(_))) } - /// Create an appropriate schema adapter based on schema characteristics. - /// Returns a NestedStructSchemaAdapter if the projected schema contains nested structs, - /// otherwise returns a DefaultSchemaAdapter. - pub fn create_adapter( - projected_table_schema: SchemaRef, - table_schema: SchemaRef, - ) -> Box { - // Use nested adapter if target has nested structs - if Self::has_nested_structs(table_schema.as_ref()) { - NestedStructSchemaAdapterFactory.create(projected_table_schema, table_schema) - } else { - // Default case for simple schemas - DefaultSchemaAdapterFactory.create(projected_table_schema, table_schema) - } - } } /// A SchemaAdapter that handles schema evolution for nested struct types @@ -449,10 +435,15 @@ mod tests { ); // Test factory selection logic - let adapter = NestedStructSchemaAdapterFactory::create_adapter( - nested_schema.clone(), - nested_schema.clone(), - ); + let adapter: Box = if + NestedStructSchemaAdapterFactory::has_nested_structs(&nested_schema) + { + NestedStructSchemaAdapterFactory + .create(nested_schema.clone(), nested_schema.clone()) + } else { + DefaultSchemaAdapterFactory + .create(nested_schema.clone(), nested_schema.clone()) + }; assert!( adapter.map_schema(&source_schema).is_ok(), From a9661e7bf4a8b641a6153c8e84b24268c792cb4e Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Wed, 11 Jun 2025 18:16:16 +0800 Subject: [PATCH 16/57] refactor: reorganize imports and simplify closure in NestedStructSchemaAdapter --- .../datasource/src/nested_schema_adapter.rs | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/datafusion/datasource/src/nested_schema_adapter.rs b/datafusion/datasource/src/nested_schema_adapter.rs index 900c32bd83952..fe2e03a566802 100644 --- a/datafusion/datasource/src/nested_schema_adapter.rs +++ b/datafusion/datasource/src/nested_schema_adapter.rs @@ -22,18 +22,22 @@ //! can be stored external to a parquet file that maps parquet logical types to arrow types. use crate::schema_adapter::{ - create_field_mapping, DefaultSchemaAdapterFactory, SchemaAdapter, - SchemaAdapterFactory, SchemaMapper, SchemaMapping, + create_field_mapping, SchemaAdapter, SchemaAdapterFactory, SchemaMapper, + SchemaMapping, }; -#[cfg(test)] use arrow::{ array::{Array, ArrayRef, StructArray}, compute::cast, datatypes::{DataType::Struct, Field, Schema, SchemaRef}, - record_batch::RecordBatch, }; -use datafusion_common::{arrow::array::new_null_array, ColumnStatistics, Result}; +use datafusion_common::{arrow::array::new_null_array, Result}; use std::sync::Arc; + +#[cfg(test)] +use { + crate::schema_adapter::DefaultSchemaAdapterFactory, arrow::record_batch::RecordBatch, + datafusion_common::ColumnStatistics, +}; /// Factory for creating [`NestedStructSchemaAdapter`] /// /// This factory creates schema adapters that properly handle schema evolution @@ -136,9 +140,7 @@ impl SchemaAdapter for NestedStructSchemaAdapter { Arc::new(SchemaMapping::new( Arc::clone(&self.projected_table_schema), field_mappings, - Arc::new(|array: &ArrayRef, field: &Field| { - Ok(adapt_column(array, field)?) - }), + Arc::new(|array: &ArrayRef, field: &Field| adapt_column(array, field)), )), projection, )) From 8b87fc6bf075b220f23109969bd8dcedad373be9 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Wed, 11 Jun 2025 20:53:50 +0800 Subject: [PATCH 17/57] refactor: enhance ListingTableConfig with Default implementation and streamline new methods --- .../core/src/datasource/listing/table.rs | 37 +++++++------------ 1 file changed, 14 insertions(+), 23 deletions(-) diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index e73dfdffedf92..ca570dbdd2479 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -68,7 +68,7 @@ pub enum SchemaSource { /// Configuration for creating a [`ListingTable`] /// /// -#[derive(Debug, Clone)] +#[derive(Debug, Clone, Default)] pub struct ListingTableConfig { /// Paths on the `ObjectStore` for creating `ListingTable`. /// They should share the same schema and object store. @@ -87,16 +87,18 @@ pub struct ListingTableConfig { schema_adapter_factory: Option>, } +impl Default for SchemaSource { + fn default() -> Self { + SchemaSource::None + } +} + impl ListingTableConfig { /// Creates new [`ListingTableConfig`] for reading the specified URL pub fn new(table_path: ListingTableUrl) -> Self { - let table_paths = vec![table_path]; Self { - table_paths, - file_schema: None, - options: None, - schema_source: SchemaSource::None, - schema_adapter_factory: None, + table_paths: vec![table_path], + ..Default::default() } } @@ -106,10 +108,7 @@ impl ListingTableConfig { pub fn new_with_multi_paths(table_paths: Vec) -> Self { Self { table_paths, - file_schema: None, - options: None, - schema_source: SchemaSource::None, - schema_adapter_factory: None, + ..Default::default() } } @@ -130,11 +129,9 @@ impl ListingTableConfig { /// without the table partitioning columns. pub fn with_schema(self, schema: SchemaRef) -> Self { Self { - table_paths: self.table_paths, file_schema: Some(schema), - options: self.options, schema_source: SchemaSource::Specified, - schema_adapter_factory: self.schema_adapter_factory, + ..self } } @@ -144,11 +141,8 @@ impl ListingTableConfig { /// [`Self::infer_options`]. pub fn with_listing_options(self, listing_options: ListingOptions) -> Self { Self { - table_paths: self.table_paths, - file_schema: self.file_schema, options: Some(listing_options), - schema_source: self.schema_source, - schema_adapter_factory: self.schema_adapter_factory, + ..self } } @@ -331,11 +325,8 @@ impl ListingTableConfig { schema_adapter_factory: Arc, ) -> Self { Self { - table_paths: self.table_paths, - file_schema: self.file_schema, - options: self.options, - schema_source: self.schema_source, schema_adapter_factory: Some(schema_adapter_factory), + ..self } } @@ -478,7 +469,7 @@ impl ListingOptions { /// the rows read from /// `/mnt/nyctaxi/year=2022/month=01/tripdata.parquet` /// - ///# Notes + /// # Notes /// /// - If only one level (e.g. `year` in the example above) is /// specified, the other levels are ignored but the files are From 71d06fe74626cf3aaf9143c1cce24a608a8b5a7d Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Wed, 11 Jun 2025 21:03:50 +0800 Subject: [PATCH 18/57] refactor: amend with_schema_adapter_factory method in ListingTable to consume self --- datafusion/core/src/datasource/listing/table.rs | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index ca570dbdd2479..a283ed9458c34 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -930,11 +930,13 @@ impl ListingTable { /// handle schema evolution and type conversions when reading files with /// different schemas than the table schema. pub fn with_schema_adapter_factory( - mut self, + self, schema_adapter_factory: Arc, ) -> Self { - self.schema_adapter_factory = Some(schema_adapter_factory); - self + Self { + schema_adapter_factory: Some(schema_adapter_factory), + ..self + } } /// Get the [`SchemaAdapterFactory`] for this table From 22ef8ad89dd9125818d6deb5420c202727aa510f Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Wed, 11 Jun 2025 22:04:24 +0800 Subject: [PATCH 19/57] refactor: add create_file_source_with_schema_adapter method to ListingTable for improved file source creation --- datafusion/core/src/datasource/listing/table.rs | 17 ++++++++++++----- 1 file changed, 12 insertions(+), 5 deletions(-) diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index a283ed9458c34..0129d560b82bc 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -39,6 +39,7 @@ use datafusion_datasource::{ file_groups::FileGroup, file_scan_config::{FileScanConfig, FileScanConfigBuilder}, schema_adapter::{DefaultSchemaAdapterFactory, SchemaAdapter, SchemaAdapterFactory}, + FileSource, }; use datafusion_execution::{ cache::{cache_manager::FileStatisticsCache, cache_unit::DefaultFileStatisticsCache}, @@ -944,6 +945,16 @@ impl ListingTable { self.schema_adapter_factory.as_ref() } + /// Creates a file source and applies schema adapter factory if available + fn create_file_source_with_schema_adapter(&self) -> Result> { + let mut source = self.options.format.file_source(); + // Apply schema adapter to source if available + if let Some(factory) = &self.schema_adapter_factory { + source = source.with_schema_adapter_factory(Arc::clone(factory))?; + } + Ok(source) + } + /// If file_sort_order is specified, creates the appropriate physical expressions fn try_create_output_ordering(&self) -> Result> { create_ordering(&self.table_schema, &self.options.file_sort_order) @@ -1052,11 +1063,7 @@ impl TableProvider for ListingTable { return Ok(Arc::new(EmptyExec::new(Arc::new(Schema::empty())))); }; - let mut source = self.options.format.file_source(); - // Apply schema adapter to source if available - if let Some(factory) = &self.schema_adapter_factory { - source = source.with_schema_adapter_factory(Arc::clone(factory))?; - } + let source = self.create_file_source_with_schema_adapter()?; // create the execution plan self.options From f99205e8c75281ccec9c04d7c0e3c269789e1bac Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Wed, 11 Jun 2025 22:12:59 +0800 Subject: [PATCH 20/57] refactor: implement FailingMapSchemaAdapterFactory for enhanced schema error handling in ListingTable --- .../core/src/datasource/listing/table.rs | 189 +++++++++++++++++- 1 file changed, 188 insertions(+), 1 deletion(-) diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 0129d560b82bc..c2cf44b7830e8 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -36,10 +36,10 @@ use datafusion_common::{ }; use datafusion_datasource::{ compute_all_files_statistics, + file::FileSource, file_groups::FileGroup, file_scan_config::{FileScanConfig, FileScanConfigBuilder}, schema_adapter::{DefaultSchemaAdapterFactory, SchemaAdapter, SchemaAdapterFactory}, - FileSource, }; use datafusion_execution::{ cache::{cache_manager::FileStatisticsCache, cache_unit::DefaultFileStatisticsCache}, @@ -2656,6 +2656,193 @@ mod tests { Ok(()) } + #[tokio::test] + async fn test_schema_adapter_map_schema_error_type_incompatible() -> Result<()> { + let ctx = SessionContext::new(); + let path = "table/file.json"; + register_test_store(&ctx, &[(path, 10)]); + + let format = JsonFormat::default(); + let opt = ListingOptions::new(Arc::new(format)); + let schema = Schema::new(vec![Field::new("a", DataType::Boolean, false)]); + let table_path = ListingTableUrl::parse("test:///table/").unwrap(); + + let config = ListingTableConfig::new(table_path) + .with_listing_options(opt) + .with_schema(Arc::new(schema)) + .with_schema_adapter_factory(Arc::new(FailingMapSchemaAdapterFactory { + error_type: MapSchemaError::TypeIncompatible, + })); + let table = ListingTable::try_new(config)?; + + // The error should bubble up from the scan operation when schema mapping fails + let scan_result = table.scan(&ctx.state(), None, &[], None).await; + + assert!(scan_result.is_err()); + let error_msg = scan_result.unwrap_err().to_string(); + assert!( + error_msg.contains("Cannot map incompatible types"), + "Expected type incompatibility error, got: {error_msg}" + ); + + Ok(()) + } + + #[tokio::test] + async fn test_schema_adapter_map_schema_error_general_failure() -> Result<()> { + let ctx = SessionContext::new(); + let path = "table/file.json"; + register_test_store(&ctx, &[(path, 10)]); + + let format = JsonFormat::default(); + let opt = ListingOptions::new(Arc::new(format)); + let schema = Schema::new(vec![Field::new("a", DataType::Boolean, false)]); + let table_path = ListingTableUrl::parse("test:///table/").unwrap(); + + let config = ListingTableConfig::new(table_path) + .with_listing_options(opt) + .with_schema(Arc::new(schema)) + .with_schema_adapter_factory(Arc::new(FailingMapSchemaAdapterFactory { + error_type: MapSchemaError::GeneralFailure, + })); + let table = ListingTable::try_new(config)?; + + // The error should bubble up from the scan operation when schema mapping fails + let scan_result = table.scan(&ctx.state(), None, &[], None).await; + + assert!(scan_result.is_err()); + let error_msg = scan_result.unwrap_err().to_string(); + assert!( + error_msg.contains("Schema adapter mapping failed"), + "Expected general failure error, got: {error_msg}" + ); + + Ok(()) + } + + #[tokio::test] + async fn test_schema_adapter_map_schema_error_invalid_projection() -> Result<()> { + let ctx = SessionContext::new(); + let path = "table/file.json"; + register_test_store(&ctx, &[(path, 10)]); + + let format = JsonFormat::default(); + let opt = ListingOptions::new(Arc::new(format)); + let schema = Schema::new(vec![Field::new("a", DataType::Boolean, false)]); + let table_path = ListingTableUrl::parse("test:///table/").unwrap(); + + let config = ListingTableConfig::new(table_path) + .with_listing_options(opt) + .with_schema(Arc::new(schema)) + .with_schema_adapter_factory(Arc::new(FailingMapSchemaAdapterFactory { + error_type: MapSchemaError::InvalidProjection, + })); + let table = ListingTable::try_new(config)?; + + // The error should bubble up from the scan operation when schema mapping fails + let scan_result = table.scan(&ctx.state(), None, &[], None).await; + + assert!(scan_result.is_err()); + let error_msg = scan_result.unwrap_err().to_string(); + assert!( + error_msg.contains("Invalid projection in schema mapping"), + "Expected invalid projection error, got: {error_msg}" + ); + + Ok(()) + } + + // Test that errors during file listing also bubble up correctly + #[tokio::test] + async fn test_schema_adapter_error_during_file_listing() -> Result<()> { + let ctx = SessionContext::new(); + let path = "table/file.json"; + register_test_store(&ctx, &[(path, 10)]); + + let format = JsonFormat::default(); + let opt = ListingOptions::new(Arc::new(format)).with_collect_stat(true); + let schema = Schema::new(vec![Field::new("a", DataType::Boolean, false)]); + let table_path = ListingTableUrl::parse("test:///table/").unwrap(); + + let config = ListingTableConfig::new(table_path) + .with_listing_options(opt) + .with_schema(Arc::new(schema)) + .with_schema_adapter_factory(Arc::new(FailingMapSchemaAdapterFactory { + error_type: MapSchemaError::TypeIncompatible, + })); + let table = ListingTable::try_new(config)?; + + // The error should bubble up from list_files_for_scan when collecting statistics + let list_result = table.list_files_for_scan(&ctx.state(), &[], None).await; + + assert!(list_result.is_err()); + let error_msg = list_result.unwrap_err().to_string(); + assert!( + error_msg.contains("Cannot map incompatible types"), + "Expected type incompatibility error during file listing, got: {error_msg}" + ); + + Ok(()) + } + + #[derive(Debug, Copy, Clone)] + enum MapSchemaError { + TypeIncompatible, + GeneralFailure, + InvalidProjection, + } + + #[derive(Debug)] + struct FailingMapSchemaAdapterFactory { + error_type: MapSchemaError, + } + + impl SchemaAdapterFactory for FailingMapSchemaAdapterFactory { + fn create( + &self, + projected_table_schema: SchemaRef, + _table_schema: SchemaRef, + ) -> Box { + Box::new(FailingMapSchemaAdapter { + schema: projected_table_schema, + error_type: self.error_type, + }) + } + } + + #[derive(Debug)] + struct FailingMapSchemaAdapter { + schema: SchemaRef, + error_type: MapSchemaError, + } + + impl SchemaAdapter for FailingMapSchemaAdapter { + fn map_column_index(&self, index: usize, file_schema: &Schema) -> Option { + let field = self.schema.field(index); + file_schema.fields.find(field.name()).map(|(i, _)| i) + } + + fn map_schema( + &self, + _file_schema: &Schema, + ) -> Result<(Arc, Vec)> { + // Always fail with different error types based on the configured error_type + match self.error_type { + MapSchemaError::TypeIncompatible => { + plan_err!( + "Cannot map incompatible types: Boolean cannot be cast to Utf8" + ) + } + MapSchemaError::GeneralFailure => { + plan_err!("Schema adapter mapping failed due to internal error") + } + MapSchemaError::InvalidProjection => { + plan_err!("Invalid projection in schema mapping: column index out of bounds") + } + } + } + } + #[derive(Debug)] struct NullStatsAdapterFactory; From 1c887d74b116a57588c521289b158345a547ea4a Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Wed, 11 Jun 2025 22:44:59 +0800 Subject: [PATCH 21/57] =?UTF-8?q?=20split=20nested=5Fschema=5Fadapter.rs?= =?UTF-8?q?=20into=20smaller=20modules=E2=80=94e.g.=20factory.rs,=20adapte?= =?UTF-8?q?r.rs,=20and=20mapping.rs=E2=80=94to=20improve=20navigability=20?= =?UTF-8?q?and=20reduce=20merge=20conflicts.?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../src/nested_schema_adapter/adapter.rs | 93 +++++++++ .../src/nested_schema_adapter/factory.rs | 61 ++++++ .../src/nested_schema_adapter/mapping.rs | 63 ++++++ .../src/nested_schema_adapter/mod.rs | 27 +++ .../tests.rs} | 181 ------------------ 5 files changed, 244 insertions(+), 181 deletions(-) create mode 100644 datafusion/datasource/src/nested_schema_adapter/adapter.rs create mode 100644 datafusion/datasource/src/nested_schema_adapter/factory.rs create mode 100644 datafusion/datasource/src/nested_schema_adapter/mapping.rs create mode 100644 datafusion/datasource/src/nested_schema_adapter/mod.rs rename datafusion/datasource/src/{nested_schema_adapter.rs => nested_schema_adapter/tests.rs} (76%) diff --git a/datafusion/datasource/src/nested_schema_adapter/adapter.rs b/datafusion/datasource/src/nested_schema_adapter/adapter.rs new file mode 100644 index 0000000000000..c9093f821d2e3 --- /dev/null +++ b/datafusion/datasource/src/nested_schema_adapter/adapter.rs @@ -0,0 +1,93 @@ +// 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 super::mapping::adapt_column; +use crate::schema_adapter::{ + create_field_mapping, SchemaAdapter, SchemaMapper, SchemaMapping, +}; +use arrow::{ + array::ArrayRef, + datatypes::{DataType::Struct, Field, Schema, SchemaRef}, +}; +use datafusion_common::Result; +use std::sync::Arc; + +/// A SchemaAdapter that handles schema evolution for nested struct types +#[derive(Debug, Clone)] +pub struct NestedStructSchemaAdapter { + /// The schema for the table, projected to include only the fields being output (projected) by the + /// associated ParquetSource + projected_table_schema: SchemaRef, + /// The entire table schema for the table we're using this to adapt. + /// + /// This is used to evaluate any filters pushed down into the scan + /// which may refer to columns that are not referred to anywhere + /// else in the plan. + table_schema: SchemaRef, +} + +impl NestedStructSchemaAdapter { + /// Create a new NestedStructSchemaAdapter with the target schema + pub fn new(projected_table_schema: SchemaRef, table_schema: SchemaRef) -> Self { + Self { + projected_table_schema, + table_schema, + } + } + + pub fn projected_table_schema(&self) -> &Schema { + self.projected_table_schema.as_ref() + } + + pub fn table_schema(&self) -> &Schema { + self.table_schema.as_ref() + } +} + +impl SchemaAdapter for NestedStructSchemaAdapter { + fn map_column_index(&self, index: usize, file_schema: &Schema) -> Option { + let field_name = self.table_schema.field(index).name(); + file_schema.index_of(field_name).ok() + } + + fn map_schema( + &self, + file_schema: &Schema, + ) -> Result<(Arc, Vec)> { + let (field_mappings, projection) = create_field_mapping( + file_schema, + &self.projected_table_schema, + |file_field, table_field| { + // Special handling for struct fields - always include them even if the + // internal structure differs, as we'll adapt them later + match (file_field.data_type(), table_field.data_type()) { + (Struct(_), Struct(_)) => Ok(true), + _ => crate::schema_adapter::can_cast_field(file_field, table_field), + } + }, + )?; + + Ok(( + Arc::new(SchemaMapping::new( + Arc::clone(&self.projected_table_schema), + field_mappings, + Arc::new(|array: &ArrayRef, field: &Field| adapt_column(array, field)), + )), + projection, + )) + } +} diff --git a/datafusion/datasource/src/nested_schema_adapter/factory.rs b/datafusion/datasource/src/nested_schema_adapter/factory.rs new file mode 100644 index 0000000000000..520c5d73dfa15 --- /dev/null +++ b/datafusion/datasource/src/nested_schema_adapter/factory.rs @@ -0,0 +1,61 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use super::adapter::NestedStructSchemaAdapter; +use crate::schema_adapter::{SchemaAdapter, SchemaAdapterFactory}; +use arrow::datatypes::{DataType::Struct, Schema, SchemaRef}; +use std::sync::Arc; + +/// Factory for creating [`NestedStructSchemaAdapter`] +/// +/// This factory creates schema adapters that properly handle schema evolution +/// for nested struct fields, allowing new fields to be added to struct columns +/// over time. +#[derive(Debug, Clone, Default)] +pub struct NestedStructSchemaAdapterFactory; + +impl SchemaAdapterFactory for NestedStructSchemaAdapterFactory { + fn create( + &self, + projected_table_schema: SchemaRef, + table_schema: SchemaRef, + ) -> Box { + Box::new(NestedStructSchemaAdapter::new( + projected_table_schema, + table_schema, + )) + } +} + +impl NestedStructSchemaAdapterFactory { + /// Create a new factory for mapping batches from a file schema to a table + /// schema with support for nested struct evolution. + /// + /// This is a convenience method that handles nested struct fields properly. + pub fn from_schema(table_schema: SchemaRef) -> Box { + Self.create(Arc::clone(&table_schema), table_schema) + } + + /// Determines if a schema contains nested struct fields that would benefit + /// from special handling during schema evolution + pub fn has_nested_structs(schema: &Schema) -> bool { + schema + .fields() + .iter() + .any(|field| matches!(field.data_type(), Struct(_))) + } +} diff --git a/datafusion/datasource/src/nested_schema_adapter/mapping.rs b/datafusion/datasource/src/nested_schema_adapter/mapping.rs new file mode 100644 index 0000000000000..7d8d2f5c7129d --- /dev/null +++ b/datafusion/datasource/src/nested_schema_adapter/mapping.rs @@ -0,0 +1,63 @@ +// 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::{ + array::{Array, ArrayRef, StructArray}, + compute::cast, + datatypes::{DataType::Struct, Field}, +}; +use datafusion_common::{arrow::array::new_null_array, Result}; +use std::sync::Arc; + +/// Adapt a column to match the target field type, handling nested structs specially +pub(super) fn adapt_column( + source_col: &ArrayRef, + target_field: &Field, +) -> Result { + match target_field.data_type() { + Struct(target_fields) => { + if let Some(struct_array) = source_col.as_any().downcast_ref::() + { + let mut children: Vec<(Arc, Arc)> = Vec::new(); + let num_rows = source_col.len(); + + for target_child_field in target_fields { + let field_arc = Arc::clone(target_child_field); + match struct_array.column_by_name(target_child_field.name()) { + Some(source_child_col) => { + let adapted_child = + adapt_column(source_child_col, target_child_field)?; + children.push((field_arc, adapted_child)); + } + None => { + children.push(( + field_arc, + new_null_array(target_child_field.data_type(), num_rows), + )); + } + } + } + + let struct_array = StructArray::from(children); + Ok(Arc::new(struct_array)) + } else { + Ok(new_null_array(target_field.data_type(), source_col.len())) + } + } + _ => Ok(cast(source_col, target_field.data_type())?), + } +} diff --git a/datafusion/datasource/src/nested_schema_adapter/mod.rs b/datafusion/datasource/src/nested_schema_adapter/mod.rs new file mode 100644 index 0000000000000..7e7c60590d34b --- /dev/null +++ b/datafusion/datasource/src/nested_schema_adapter/mod.rs @@ -0,0 +1,27 @@ +// 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. + +//! Nested schema adaptation utilities. + +pub mod adapter; +pub mod factory; +mod mapping; +#[cfg(test)] +mod tests; + +pub use adapter::NestedStructSchemaAdapter; +pub use factory::NestedStructSchemaAdapterFactory; diff --git a/datafusion/datasource/src/nested_schema_adapter.rs b/datafusion/datasource/src/nested_schema_adapter/tests.rs similarity index 76% rename from datafusion/datasource/src/nested_schema_adapter.rs rename to datafusion/datasource/src/nested_schema_adapter/tests.rs index fe2e03a566802..ce647044129d5 100644 --- a/datafusion/datasource/src/nested_schema_adapter.rs +++ b/datafusion/datasource/src/nested_schema_adapter/tests.rs @@ -15,187 +15,6 @@ // specific language governing permissions and limitations // under the License. -//! [`NestedStructSchemaAdapter`] and [`NestedStructSchemaAdapterFactory`] to adapt file-level record batches to a table schema. -//! -//! Adapter provides a method of translating the RecordBatches that come out of the -//! physical format into how they should be used by DataFusion. For instance, a schema -//! can be stored external to a parquet file that maps parquet logical types to arrow types. - -use crate::schema_adapter::{ - create_field_mapping, SchemaAdapter, SchemaAdapterFactory, SchemaMapper, - SchemaMapping, -}; -use arrow::{ - array::{Array, ArrayRef, StructArray}, - compute::cast, - datatypes::{DataType::Struct, Field, Schema, SchemaRef}, -}; -use datafusion_common::{arrow::array::new_null_array, Result}; -use std::sync::Arc; - -#[cfg(test)] -use { - crate::schema_adapter::DefaultSchemaAdapterFactory, arrow::record_batch::RecordBatch, - datafusion_common::ColumnStatistics, -}; -/// Factory for creating [`NestedStructSchemaAdapter`] -/// -/// This factory creates schema adapters that properly handle schema evolution -/// for nested struct fields, allowing new fields to be added to struct columns -/// over time. -#[derive(Debug, Clone, Default)] -pub struct NestedStructSchemaAdapterFactory; - -impl SchemaAdapterFactory for NestedStructSchemaAdapterFactory { - fn create( - &self, - projected_table_schema: SchemaRef, - table_schema: SchemaRef, - ) -> Box { - Box::new(NestedStructSchemaAdapter::new( - projected_table_schema, - table_schema, - )) - } -} - -impl NestedStructSchemaAdapterFactory { - /// Create a new factory for mapping batches from a file schema to a table - /// schema with support for nested struct evolution. - /// - /// This is a convenience method that handles nested struct fields properly. - pub fn from_schema(table_schema: SchemaRef) -> Box { - Self.create(Arc::clone(&table_schema), table_schema) - } - - /// Determines if a schema contains nested struct fields that would benefit - /// from special handling during schema evolution - pub fn has_nested_structs(schema: &Schema) -> bool { - schema - .fields() - .iter() - .any(|field| matches!(field.data_type(), Struct(_))) - } -} - -/// A SchemaAdapter that handles schema evolution for nested struct types -#[derive(Debug, Clone)] -pub struct NestedStructSchemaAdapter { - /// The schema for the table, projected to include only the fields being output (projected) by the - /// associated ParquetSource - projected_table_schema: SchemaRef, - /// The entire table schema for the table we're using this to adapt. - /// - /// This is used to evaluate any filters pushed down into the scan - /// which may refer to columns that are not referred to anywhere - /// else in the plan. - table_schema: SchemaRef, -} - -impl NestedStructSchemaAdapter { - /// Create a new NestedStructSchemaAdapter with the target schema - pub fn new(projected_table_schema: SchemaRef, table_schema: SchemaRef) -> Self { - Self { - projected_table_schema, - table_schema, - } - } - - pub fn projected_table_schema(&self) -> &Schema { - self.projected_table_schema.as_ref() - } - - pub fn table_schema(&self) -> &Schema { - self.table_schema.as_ref() - } -} - -impl SchemaAdapter for NestedStructSchemaAdapter { - fn map_column_index(&self, index: usize, file_schema: &Schema) -> Option { - let field_name = self.table_schema.field(index).name(); - file_schema.index_of(field_name).ok() - } - - fn map_schema( - &self, - file_schema: &Schema, - ) -> Result<(Arc, Vec)> { - let (field_mappings, projection) = create_field_mapping( - file_schema, - &self.projected_table_schema, - |file_field, table_field| { - // Special handling for struct fields - always include them even if the - // internal structure differs, as we'll adapt them later - match (file_field.data_type(), table_field.data_type()) { - (Struct(_), Struct(_)) => Ok(true), - _ => { - // For non-struct fields, use the regular cast check - crate::schema_adapter::can_cast_field(file_field, table_field) - } - } - }, - )?; - - Ok(( - Arc::new(SchemaMapping::new( - Arc::clone(&self.projected_table_schema), - field_mappings, - Arc::new(|array: &ArrayRef, field: &Field| adapt_column(array, field)), - )), - projection, - )) - } -} - -// Helper methods for nested struct adaptation -/// Adapt a column to match the target field type, handling nested structs specially -fn adapt_column(source_col: &ArrayRef, target_field: &Field) -> Result { - match target_field.data_type() { - Struct(target_fields) => { - // For struct arrays, we need to handle them specially - if let Some(struct_array) = source_col.as_any().downcast_ref::() - { - // Create a vector to store field-array pairs with the correct type - let mut children: Vec<(Arc, Arc)> = Vec::new(); - let num_rows = source_col.len(); - - // For each field in the target schema - for target_child_field in target_fields { - // Create Arc directly (not Arc>) - let field_arc = Arc::clone(target_child_field); - - // Try to find corresponding field in source - match struct_array.column_by_name(target_child_field.name()) { - Some(source_child_col) => { - // Field exists in source, adapt it - let adapted_child = - adapt_column(source_child_col, target_child_field)?; - children.push((field_arc, adapted_child)); - } - None => { - // Field doesn't exist in source, add null array - children.push(( - field_arc, - new_null_array(target_child_field.data_type(), num_rows), - )); - } - } - } - - // Create new struct array with all target fields - let struct_array = StructArray::from(children); - Ok(Arc::new(struct_array)) - } else { - // Not a struct array, but target expects struct - return nulls - Ok(new_null_array(target_field.data_type(), source_col.len())) - } - } - // For non-struct types, just cast - _ => Ok(cast(source_col, target_field.data_type())?), - } -} - -#[cfg(test)] mod tests { use super::*; use arrow::{ From e4913d1cddac41b0ab4567ff7255b19c956588da Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Wed, 11 Jun 2025 22:50:48 +0800 Subject: [PATCH 22/57] refactor: reorder imports in tests module for improved clarity --- datafusion/core/src/datasource/listing/table.rs | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index c2cf44b7830e8..b78dab97c1ee0 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -1385,12 +1385,11 @@ mod tests { }, }; use arrow::{compute::SortOptions, record_batch::RecordBatch}; - use datafusion_common::ColumnStatistics; use datafusion_common::{ assert_contains, stats::Precision, test_util::{batches_to_string, datafusion_test_data}, - ScalarValue, + ColumnStatistics, ScalarValue, }; use datafusion_datasource::schema_adapter::{ SchemaAdapter, SchemaAdapterFactory, SchemaMapper, @@ -1401,7 +1400,6 @@ mod tests { use std::io::Write; use tempfile::TempDir; use url::Url; - /// Creates a test schema with standard field types used in tests fn create_test_schema() -> SchemaRef { Arc::new(Schema::new(vec![ From e70ce7960eb07751befb697e25c01db90699cc47 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Wed, 11 Jun 2025 22:53:23 +0800 Subject: [PATCH 23/57] refactor: enhance adapt_column and adapt_struct_column for improved clarity and functionality --- .../src/nested_schema_adapter/mapping.rs | 67 +++++++++++-------- 1 file changed, 38 insertions(+), 29 deletions(-) diff --git a/datafusion/datasource/src/nested_schema_adapter/mapping.rs b/datafusion/datasource/src/nested_schema_adapter/mapping.rs index 7d8d2f5c7129d..3b22ef65e0024 100644 --- a/datafusion/datasource/src/nested_schema_adapter/mapping.rs +++ b/datafusion/datasource/src/nested_schema_adapter/mapping.rs @@ -23,41 +23,50 @@ use arrow::{ use datafusion_common::{arrow::array::new_null_array, Result}; use std::sync::Arc; -/// Adapt a column to match the target field type, handling nested structs specially -pub(super) fn adapt_column( +/// Adapt a struct column to match the target field type, handling nested structs recursively +fn adapt_struct_column( source_col: &ArrayRef, - target_field: &Field, + target_fields: &[Arc], ) -> Result { - match target_field.data_type() { - Struct(target_fields) => { - if let Some(struct_array) = source_col.as_any().downcast_ref::() - { - let mut children: Vec<(Arc, Arc)> = Vec::new(); - let num_rows = source_col.len(); + if let Some(struct_array) = source_col.as_any().downcast_ref::() { + let mut children: Vec<(Arc, Arc)> = Vec::new(); + let num_rows = source_col.len(); - for target_child_field in target_fields { - let field_arc = Arc::clone(target_child_field); - match struct_array.column_by_name(target_child_field.name()) { - Some(source_child_col) => { - let adapted_child = - adapt_column(source_child_col, target_child_field)?; - children.push((field_arc, adapted_child)); - } - None => { - children.push(( - field_arc, - new_null_array(target_child_field.data_type(), num_rows), - )); - } - } + for target_child_field in target_fields { + let field_arc = Arc::clone(target_child_field); + match struct_array.column_by_name(target_child_field.name()) { + Some(source_child_col) => { + let adapted_child = + adapt_column(source_child_col, target_child_field)?; + children.push((field_arc, adapted_child)); + } + None => { + children.push(( + field_arc, + new_null_array(target_child_field.data_type(), num_rows), + )); } - - let struct_array = StructArray::from(children); - Ok(Arc::new(struct_array)) - } else { - Ok(new_null_array(target_field.data_type(), source_col.len())) } } + + let struct_array = StructArray::from(children); + Ok(Arc::new(struct_array)) + } else { + // If source is not a struct, return null array with target struct type + Ok(new_null_array( + &Struct(target_fields.to_vec().into()), + source_col.len(), + )) + } +} + +/// Adapt a column to match the target field type, handling nested structs specially +pub(super) fn adapt_column( + source_col: &ArrayRef, + target_field: &Field, +) -> Result { + match target_field.data_type() { + Struct(target_fields) => adapt_struct_column(source_col, target_fields), _ => Ok(cast(source_col, target_field.data_type())?), } } From dafb68e5eeb8358a48b7172240a0b4909fea4611 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Wed, 11 Jun 2025 22:57:46 +0800 Subject: [PATCH 24/57] refactor: impl Debug for SchemaMapping adapt_column --- datafusion/datasource/src/schema_adapter.rs | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/datafusion/datasource/src/schema_adapter.rs b/datafusion/datasource/src/schema_adapter.rs index 6dbba2110c254..b8fc9b19d5e01 100644 --- a/datafusion/datasource/src/schema_adapter.rs +++ b/datafusion/datasource/src/schema_adapter.rs @@ -282,7 +282,9 @@ impl SchemaAdapter for DefaultSchemaAdapter { Arc::new(SchemaMapping::new( Arc::clone(&self.projected_table_schema), field_mappings, - Arc::new(|array: &ArrayRef, field: &Field| Ok(cast(array, field.data_type())?)), + Arc::new(|array: &ArrayRef, field: &Field| { + Ok(cast(array, field.data_type())?) + }), )), projection, )) @@ -349,6 +351,7 @@ impl Debug for SchemaMapping { f.debug_struct("SchemaMapping") .field("projected_table_schema", &self.projected_table_schema) .field("field_mappings", &self.field_mappings) + .field("adapt_column", &"") .finish() } } @@ -621,7 +624,9 @@ mod tests { let mapping = SchemaMapping::new( Arc::clone(&projected_schema), field_mappings.clone(), - Arc::new(|array: &ArrayRef, field: &Field| Ok(cast(array, field.data_type())?)), + Arc::new(|array: &ArrayRef, field: &Field| { + Ok(cast(array, field.data_type())?) + }), ); // Check that fields were set correctly From fd8929c28086d1b3d4a26a2e1c7bc86b55af60de Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 12 Jun 2025 09:42:54 +0800 Subject: [PATCH 25/57] refactor: update imports in tests module for clarity and organization --- .../src/nested_schema_adapter/tests.rs | 19 +++++++++++++++---- 1 file changed, 15 insertions(+), 4 deletions(-) diff --git a/datafusion/datasource/src/nested_schema_adapter/tests.rs b/datafusion/datasource/src/nested_schema_adapter/tests.rs index ce647044129d5..970402bb0b0e3 100644 --- a/datafusion/datasource/src/nested_schema_adapter/tests.rs +++ b/datafusion/datasource/src/nested_schema_adapter/tests.rs @@ -16,13 +16,24 @@ // under the License. mod tests { - use super::*; + use crate::nested_schema_adapter::{ + NestedStructSchemaAdapter, NestedStructSchemaAdapterFactory, + }; + use crate::schema_adapter::{ + DefaultSchemaAdapterFactory, SchemaAdapter, SchemaAdapterFactory, + }; use arrow::{ array::{Array, StringBuilder, StructArray, TimestampMillisecondArray}, - datatypes::DataType::{Float64, Int16, Int32, Timestamp, Utf8}, - datatypes::TimeUnit::Millisecond, + compute::cast, + datatypes::{ + DataType::{Float64, Int16, Int32, Struct, Timestamp, Utf8}, + Field, Schema, SchemaRef, + TimeUnit::Millisecond, + }, + record_batch::RecordBatch, }; - use datafusion_common::ScalarValue; + use datafusion_common::{ColumnStatistics, Result, ScalarValue}; + use std::sync::Arc; // ================================ // Schema Creation Helper Functions From 034716ffef53a79dce23b11879d66d8b3029e0e0 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 12 Jun 2025 11:24:45 +0800 Subject: [PATCH 26/57] refactor: rename source to file_source for clarity in TableProvider implementation --- datafusion/core/src/datasource/listing/table.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index b78dab97c1ee0..d4b4c79850960 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -1063,7 +1063,7 @@ impl TableProvider for ListingTable { return Ok(Arc::new(EmptyExec::new(Arc::new(Schema::empty())))); }; - let source = self.create_file_source_with_schema_adapter()?; + let file_source = self.create_file_source_with_schema_adapter()?; // create the execution plan self.options @@ -1073,7 +1073,7 @@ impl TableProvider for ListingTable { FileScanConfigBuilder::new( object_store_url, Arc::clone(&self.file_schema), - source, + file_source, ) .with_file_groups(partitioned_file_lists) .with_constraints(self.constraints.clone()) From 2413723916aebeb52834b82032e3c966b29fe03d Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 12 Jun 2025 11:33:36 +0800 Subject: [PATCH 27/57] refactor: extract schema adapter creation into a separate method for improved clarity --- .../core/src/datasource/listing/table.rs | 24 +++++++++++++------ 1 file changed, 17 insertions(+), 7 deletions(-) diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index d4b4c79850960..a2d8c11ef1cb0 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -945,6 +945,20 @@ impl ListingTable { self.schema_adapter_factory.as_ref() } + /// Creates a schema adapter for mapping between file and table schemas + /// + /// Uses the configured schema adapter factory if available, otherwise falls back + /// to the default implementation. + fn create_schema_adapter(&self) -> Box { + let table_schema = self.schema(); + match &self.schema_adapter_factory { + Some(factory) => { + factory.create(Arc::clone(&table_schema), Arc::clone(&table_schema)) + } + None => DefaultSchemaAdapterFactory::from_schema(Arc::clone(&table_schema)), + } + } + /// Creates a file source and applies schema adapter factory if available fn create_file_source_with_schema_adapter(&self) -> Result> { let mut source = self.options.format.file_source(); @@ -1232,14 +1246,10 @@ impl ListingTable { self.options.collect_stat, inexact_stats, )?; - let table_schema = self.schema(); - let schema_adapter: Box = match &self.schema_adapter_factory { - Some(factory) => { - factory.create(Arc::clone(&table_schema), Arc::clone(&table_schema)) - } - None => DefaultSchemaAdapterFactory::from_schema(Arc::clone(&table_schema)), - }; + + let schema_adapter = self.create_schema_adapter(); let (schema_mapper, _) = schema_adapter.map_schema(self.file_schema.as_ref())?; + stats.column_statistics = schema_mapper.map_column_statistics(&stats.column_statistics)?; file_groups.iter_mut().try_for_each(|file_group| { From 85d41da008701b01012476d266175b61e51e418d Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 12 Jun 2025 13:32:10 +0800 Subject: [PATCH 28/57] refactor: simplify test setup by introducing a helper function for creating ListingTable with JSON format and custom schema adapter --- .../core/src/datasource/listing/table.rs | 121 ++++++++---------- 1 file changed, 52 insertions(+), 69 deletions(-) diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index a2d8c11ef1cb0..d0288b3d67588 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -1410,6 +1410,7 @@ mod tests { use std::io::Write; use tempfile::TempDir; use url::Url; + /// Creates a test schema with standard field types used in tests fn create_test_schema() -> SchemaRef { Arc::new(Schema::new(vec![ @@ -2638,19 +2639,11 @@ mod tests { #[tokio::test] async fn test_statistics_mapping_with_custom_factory() -> Result<()> { let ctx = SessionContext::new(); - let path = "table/file.json"; - register_test_store(&ctx, &[(path, 10)]); - - let format = JsonFormat::default(); - let opt = ListingOptions::new(Arc::new(format)).with_collect_stat(false); - let schema = Schema::new(vec![Field::new("a", DataType::Boolean, false)]); - let table_path = ListingTableUrl::parse("test:///table/").unwrap(); - - let config = ListingTableConfig::new(table_path) - .with_listing_options(opt) - .with_schema(Arc::new(schema)) - .with_schema_adapter_factory(Arc::new(NullStatsAdapterFactory {})); - let table = ListingTable::try_new(config)?; + let table = create_test_listing_table_with_json_and_adapter( + &ctx, + false, + Arc::new(NullStatsAdapterFactory {}), + )?; let (groups, stats) = table.list_files_for_scan(&ctx.state(), &[], None).await?; @@ -2667,21 +2660,13 @@ mod tests { #[tokio::test] async fn test_schema_adapter_map_schema_error_type_incompatible() -> Result<()> { let ctx = SessionContext::new(); - let path = "table/file.json"; - register_test_store(&ctx, &[(path, 10)]); - - let format = JsonFormat::default(); - let opt = ListingOptions::new(Arc::new(format)); - let schema = Schema::new(vec![Field::new("a", DataType::Boolean, false)]); - let table_path = ListingTableUrl::parse("test:///table/").unwrap(); - - let config = ListingTableConfig::new(table_path) - .with_listing_options(opt) - .with_schema(Arc::new(schema)) - .with_schema_adapter_factory(Arc::new(FailingMapSchemaAdapterFactory { + let table = create_test_listing_table_with_json_and_adapter( + &ctx, + false, + Arc::new(FailingMapSchemaAdapterFactory { error_type: MapSchemaError::TypeIncompatible, - })); - let table = ListingTable::try_new(config)?; + }), + )?; // The error should bubble up from the scan operation when schema mapping fails let scan_result = table.scan(&ctx.state(), None, &[], None).await; @@ -2699,21 +2684,13 @@ mod tests { #[tokio::test] async fn test_schema_adapter_map_schema_error_general_failure() -> Result<()> { let ctx = SessionContext::new(); - let path = "table/file.json"; - register_test_store(&ctx, &[(path, 10)]); - - let format = JsonFormat::default(); - let opt = ListingOptions::new(Arc::new(format)); - let schema = Schema::new(vec![Field::new("a", DataType::Boolean, false)]); - let table_path = ListingTableUrl::parse("test:///table/").unwrap(); - - let config = ListingTableConfig::new(table_path) - .with_listing_options(opt) - .with_schema(Arc::new(schema)) - .with_schema_adapter_factory(Arc::new(FailingMapSchemaAdapterFactory { + let table = create_test_listing_table_with_json_and_adapter( + &ctx, + false, + Arc::new(FailingMapSchemaAdapterFactory { error_type: MapSchemaError::GeneralFailure, - })); - let table = ListingTable::try_new(config)?; + }), + )?; // The error should bubble up from the scan operation when schema mapping fails let scan_result = table.scan(&ctx.state(), None, &[], None).await; @@ -2731,21 +2708,13 @@ mod tests { #[tokio::test] async fn test_schema_adapter_map_schema_error_invalid_projection() -> Result<()> { let ctx = SessionContext::new(); - let path = "table/file.json"; - register_test_store(&ctx, &[(path, 10)]); - - let format = JsonFormat::default(); - let opt = ListingOptions::new(Arc::new(format)); - let schema = Schema::new(vec![Field::new("a", DataType::Boolean, false)]); - let table_path = ListingTableUrl::parse("test:///table/").unwrap(); - - let config = ListingTableConfig::new(table_path) - .with_listing_options(opt) - .with_schema(Arc::new(schema)) - .with_schema_adapter_factory(Arc::new(FailingMapSchemaAdapterFactory { + let table = create_test_listing_table_with_json_and_adapter( + &ctx, + false, + Arc::new(FailingMapSchemaAdapterFactory { error_type: MapSchemaError::InvalidProjection, - })); - let table = ListingTable::try_new(config)?; + }), + )?; // The error should bubble up from the scan operation when schema mapping fails let scan_result = table.scan(&ctx.state(), None, &[], None).await; @@ -2764,21 +2733,13 @@ mod tests { #[tokio::test] async fn test_schema_adapter_error_during_file_listing() -> Result<()> { let ctx = SessionContext::new(); - let path = "table/file.json"; - register_test_store(&ctx, &[(path, 10)]); - - let format = JsonFormat::default(); - let opt = ListingOptions::new(Arc::new(format)).with_collect_stat(true); - let schema = Schema::new(vec![Field::new("a", DataType::Boolean, false)]); - let table_path = ListingTableUrl::parse("test:///table/").unwrap(); - - let config = ListingTableConfig::new(table_path) - .with_listing_options(opt) - .with_schema(Arc::new(schema)) - .with_schema_adapter_factory(Arc::new(FailingMapSchemaAdapterFactory { + let table = create_test_listing_table_with_json_and_adapter( + &ctx, + true, + Arc::new(FailingMapSchemaAdapterFactory { error_type: MapSchemaError::TypeIncompatible, - })); - let table = ListingTable::try_new(config)?; + }), + )?; // The error should bubble up from list_files_for_scan when collecting statistics let list_result = table.list_files_for_scan(&ctx.state(), &[], None).await; @@ -2908,4 +2869,26 @@ mod tests { .collect()) } } + + /// Helper function to create a test ListingTable with JSON format and custom schema adapter factory + fn create_test_listing_table_with_json_and_adapter( + ctx: &SessionContext, + collect_stat: bool, + schema_adapter_factory: Arc, + ) -> Result { + let path = "table/file.json"; + register_test_store(ctx, &[(path, 10)]); + + let format = JsonFormat::default(); + let opt = ListingOptions::new(Arc::new(format)).with_collect_stat(collect_stat); + let schema = Schema::new(vec![Field::new("a", DataType::Boolean, false)]); + let table_path = ListingTableUrl::parse("test:///table/").unwrap(); + + let config = ListingTableConfig::new(table_path) + .with_listing_options(opt) + .with_schema(Arc::new(schema)) + .with_schema_adapter_factory(schema_adapter_factory); + + ListingTable::try_new(config) + } } From 6a25f2744bf88353dbe2eede4618f7c56e91fea3 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 12 Jun 2025 13:45:56 +0800 Subject: [PATCH 29/57] test: replace magic number with constant for null count in tests --- datafusion/core/src/datasource/listing/table.rs | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index d0288b3d67588..87a3a5e373e26 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -1411,6 +1411,8 @@ mod tests { use tempfile::TempDir; use url::Url; + const DUMMY_NULL_COUNT = Precision::Exact(42); + /// Creates a test schema with standard field types used in tests fn create_test_schema() -> SchemaRef { Arc::new(Schema::new(vec![ @@ -2647,10 +2649,10 @@ mod tests { let (groups, stats) = table.list_files_for_scan(&ctx.state(), &[], None).await?; - assert_eq!(stats.column_statistics[0].null_count, Precision::Exact(42)); + assert_eq!(stats.column_statistics[0].null_count, DUMMY_NULL_COUNT); for g in groups { if let Some(s) = g.file_statistics(None) { - assert_eq!(s.column_statistics[0].null_count, Precision::Exact(42)); + assert_eq!(s.column_statistics[0].null_count, DUMMY_NULL_COUNT); } } @@ -2863,7 +2865,7 @@ mod tests { .iter() .map(|s| { let mut s = s.clone(); - s.null_count = Precision::Exact(42); + s.null_count = DUMMY_NULL_COUNT); s }) .collect()) From d42bd3483e5dad252d4577e74387d71f8979c5af Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 12 Jun 2025 14:05:39 +0800 Subject: [PATCH 30/57] fix: specify type for DUMMY_NULL_COUNT constant in tests for clarity --- datafusion/core/src/datasource/listing/table.rs | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 87a3a5e373e26..dd2b2f7c93bfc 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -1411,7 +1411,7 @@ mod tests { use tempfile::TempDir; use url::Url; - const DUMMY_NULL_COUNT = Precision::Exact(42); + const DUMMY_NULL_COUNT: Precision = Precision::Exact(42); /// Creates a test schema with standard field types used in tests fn create_test_schema() -> SchemaRef { @@ -2649,6 +2649,7 @@ mod tests { let (groups, stats) = table.list_files_for_scan(&ctx.state(), &[], None).await?; + // NullStatsAdapterFactory sets null_counts to DUMMY_NULL_COUNT assert_eq!(stats.column_statistics[0].null_count, DUMMY_NULL_COUNT); for g in groups { if let Some(s) = g.file_statistics(None) { @@ -2865,7 +2866,7 @@ mod tests { .iter() .map(|s| { let mut s = s.clone(); - s.null_count = DUMMY_NULL_COUNT); + s.null_count = DUMMY_NULL_COUNT; s }) .collect()) From 1140ce7bd3b5542b6322b1ef37b5ae9387d5ff9c Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 12 Jun 2025 14:11:17 +0800 Subject: [PATCH 31/57] fix: update DUMMY_NULL_COUNT constant type from u8 to usize for accuracy --- datafusion/core/src/datasource/listing/table.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index dd2b2f7c93bfc..f7bc60045193a 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -1411,7 +1411,7 @@ mod tests { use tempfile::TempDir; use url::Url; - const DUMMY_NULL_COUNT: Precision = Precision::Exact(42); + const DUMMY_NULL_COUNT: Precision = Precision::Exact(42); /// Creates a test schema with standard field types used in tests fn create_test_schema() -> SchemaRef { From c48a43f55093364a466953c14036b3ec80d7455c Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 12 Jun 2025 14:19:23 +0800 Subject: [PATCH 32/57] test: refactor schema adapter error tests to use rstest for improved readability --- .../core/src/datasource/listing/table.rs | 74 +++++-------------- 1 file changed, 17 insertions(+), 57 deletions(-) diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index f7bc60045193a..70b1e1a9299e0 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -117,7 +117,6 @@ impl ListingTableConfig { pub fn schema_source(&self) -> SchemaSource { self.schema_source } - /// Set the `schema` for the overall [`ListingTable`] /// /// [`ListingTable`] will automatically coerce, when possible, the schema @@ -470,7 +469,7 @@ impl ListingOptions { /// the rows read from /// `/mnt/nyctaxi/year=2022/month=01/tripdata.parquet` /// - /// # Notes + ///# Notes /// /// - If only one level (e.g. `year` in the example above) is /// specified, the other levels are ignored but the files are @@ -1407,6 +1406,7 @@ mod tests { use datafusion_expr::{BinaryExpr, LogicalPlanBuilder, Operator}; use datafusion_physical_expr::PhysicalSortExpr; use datafusion_physical_plan::{collect, ExecutionPlanProperties}; + use rstest::rstest; use std::io::Write; use tempfile::TempDir; use url::Url; @@ -2649,7 +2649,7 @@ mod tests { let (groups, stats) = table.list_files_for_scan(&ctx.state(), &[], None).await?; - // NullStatsAdapterFactory sets null_counts to DUMMY_NULL_COUNT + // NullStatsAdapterFactory sets column_statistics null_count to DUMMY_NULL_COUNT assert_eq!(stats.column_statistics[0].null_count, DUMMY_NULL_COUNT); for g in groups { if let Some(s) = g.file_statistics(None) { @@ -2660,63 +2660,23 @@ mod tests { Ok(()) } + #[rstest] + #[case(MapSchemaError::TypeIncompatible, "Cannot map incompatible types")] + #[case(MapSchemaError::GeneralFailure, "Schema adapter mapping failed")] + #[case( + MapSchemaError::InvalidProjection, + "Invalid projection in schema mapping" + )] #[tokio::test] - async fn test_schema_adapter_map_schema_error_type_incompatible() -> Result<()> { - let ctx = SessionContext::new(); - let table = create_test_listing_table_with_json_and_adapter( - &ctx, - false, - Arc::new(FailingMapSchemaAdapterFactory { - error_type: MapSchemaError::TypeIncompatible, - }), - )?; - - // The error should bubble up from the scan operation when schema mapping fails - let scan_result = table.scan(&ctx.state(), None, &[], None).await; - - assert!(scan_result.is_err()); - let error_msg = scan_result.unwrap_err().to_string(); - assert!( - error_msg.contains("Cannot map incompatible types"), - "Expected type incompatibility error, got: {error_msg}" - ); - - Ok(()) - } - - #[tokio::test] - async fn test_schema_adapter_map_schema_error_general_failure() -> Result<()> { - let ctx = SessionContext::new(); - let table = create_test_listing_table_with_json_and_adapter( - &ctx, - false, - Arc::new(FailingMapSchemaAdapterFactory { - error_type: MapSchemaError::GeneralFailure, - }), - )?; - - // The error should bubble up from the scan operation when schema mapping fails - let scan_result = table.scan(&ctx.state(), None, &[], None).await; - - assert!(scan_result.is_err()); - let error_msg = scan_result.unwrap_err().to_string(); - assert!( - error_msg.contains("Schema adapter mapping failed"), - "Expected general failure error, got: {error_msg}" - ); - - Ok(()) - } - - #[tokio::test] - async fn test_schema_adapter_map_schema_error_invalid_projection() -> Result<()> { + async fn test_schema_adapter_map_schema_errors( + #[case] error_type: MapSchemaError, + #[case] expected_error_msg: &str, + ) -> Result<()> { let ctx = SessionContext::new(); let table = create_test_listing_table_with_json_and_adapter( &ctx, false, - Arc::new(FailingMapSchemaAdapterFactory { - error_type: MapSchemaError::InvalidProjection, - }), + Arc::new(FailingMapSchemaAdapterFactory { error_type }), )?; // The error should bubble up from the scan operation when schema mapping fails @@ -2725,8 +2685,8 @@ mod tests { assert!(scan_result.is_err()); let error_msg = scan_result.unwrap_err().to_string(); assert!( - error_msg.contains("Invalid projection in schema mapping"), - "Expected invalid projection error, got: {error_msg}" + error_msg.contains(expected_error_msg), + "Expected error containing '{expected_error_msg}', got: {error_msg}" ); Ok(()) From fc85f36ce78d84d70cf8f5341a40547cf8dfd589 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 12 Jun 2025 15:24:06 +0800 Subject: [PATCH 33/57] fix: update comment for clarity on DUMMY_NULL_COUNT usage in tests --- datafusion/core/src/datasource/listing/table.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 70b1e1a9299e0..e5810186c7ad9 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -2644,12 +2644,12 @@ mod tests { let table = create_test_listing_table_with_json_and_adapter( &ctx, false, + // NullStatsAdapterFactory sets column_statistics null_count to DUMMY_NULL_COUNT Arc::new(NullStatsAdapterFactory {}), )?; let (groups, stats) = table.list_files_for_scan(&ctx.state(), &[], None).await?; - // NullStatsAdapterFactory sets column_statistics null_count to DUMMY_NULL_COUNT assert_eq!(stats.column_statistics[0].null_count, DUMMY_NULL_COUNT); for g in groups { if let Some(s) = g.file_statistics(None) { From 9b078ce6570f6cf81a83763c11fb8b69781ed7a8 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 12 Jun 2025 15:24:13 +0800 Subject: [PATCH 34/57] refactor: organize imports and clean up test module structure --- datafusion/datasource/src/schema_adapter.rs | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/datafusion/datasource/src/schema_adapter.rs b/datafusion/datasource/src/schema_adapter.rs index b8fc9b19d5e01..5195f6e6079e6 100644 --- a/datafusion/datasource/src/schema_adapter.rs +++ b/datafusion/datasource/src/schema_adapter.rs @@ -20,14 +20,13 @@ //! Adapter provides a method of translating the RecordBatches that come out of the //! physical format into how they should be used by DataFusion. For instance, a schema //! can be stored external to a parquet file that maps parquet logical types to arrow types. - -use arrow::array::{new_null_array, ArrayRef, RecordBatch, RecordBatchOptions}; -use arrow::compute::{can_cast_types, cast}; -use arrow::datatypes::{Field, Schema, SchemaRef}; +use arrow::{ + array::{new_null_array, ArrayRef, RecordBatch, RecordBatchOptions}, + compute::{can_cast_types, cast}, + datatypes::{Field, Schema, SchemaRef}, +}; use datafusion_common::{plan_err, ColumnStatistics}; -use std::fmt::Debug; -use std::sync::Arc; - +use std::{fmt::Debug, sync::Arc}; /// Function used by [`SchemaMapping`] to adapt a column from the file schema to /// the table schema. pub type AdaptColumnFn = @@ -443,6 +442,7 @@ impl SchemaMapper for SchemaMapping { #[cfg(test)] mod tests { + use super::*; use arrow::{ array::ArrayRef, compute::cast, @@ -450,8 +450,6 @@ mod tests { }; use datafusion_common::{stats::Precision, Statistics}; - use super::*; - #[test] fn test_schema_mapping_map_statistics_basic() { // Create table schema (a, b, c) From fd8cb10ff57e3e8649f911c98f1186028d17d24a Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 12 Jun 2025 15:27:12 +0800 Subject: [PATCH 35/57] feat: add nested_struct module and adapt_column function for handling nested structs --- datafusion/common/src/lib.rs | 1 + .../mapping.rs => common/src/nested_struct.rs} | 8 +++----- .../datasource/src/nested_schema_adapter/adapter.rs | 2 +- datafusion/datasource/src/nested_schema_adapter/mod.rs | 1 - 4 files changed, 5 insertions(+), 7 deletions(-) rename datafusion/{datasource/src/nested_schema_adapter/mapping.rs => common/src/nested_struct.rs} (93%) diff --git a/datafusion/common/src/lib.rs b/datafusion/common/src/lib.rs index 7b2c86d3975ff..19152b682ec3a 100644 --- a/datafusion/common/src/lib.rs +++ b/datafusion/common/src/lib.rs @@ -50,6 +50,7 @@ pub mod parsers; pub mod pruning; pub mod rounding; pub mod scalar; +pub mod nested_struct; pub mod spans; pub mod stats; pub mod test_util; diff --git a/datafusion/datasource/src/nested_schema_adapter/mapping.rs b/datafusion/common/src/nested_struct.rs similarity index 93% rename from datafusion/datasource/src/nested_schema_adapter/mapping.rs rename to datafusion/common/src/nested_struct.rs index 3b22ef65e0024..bb2a93767a37e 100644 --- a/datafusion/datasource/src/nested_schema_adapter/mapping.rs +++ b/datafusion/common/src/nested_struct.rs @@ -15,12 +15,13 @@ // specific language governing permissions and limitations // under the License. +use crate::error::Result; +use arrow::array::new_null_array; use arrow::{ array::{Array, ArrayRef, StructArray}, compute::cast, datatypes::{DataType::Struct, Field}, }; -use datafusion_common::{arrow::array::new_null_array, Result}; use std::sync::Arc; /// Adapt a struct column to match the target field type, handling nested structs recursively @@ -61,10 +62,7 @@ fn adapt_struct_column( } /// Adapt a column to match the target field type, handling nested structs specially -pub(super) fn adapt_column( - source_col: &ArrayRef, - target_field: &Field, -) -> Result { +pub fn adapt_column(source_col: &ArrayRef, target_field: &Field) -> Result { match target_field.data_type() { Struct(target_fields) => adapt_struct_column(source_col, target_fields), _ => Ok(cast(source_col, target_field.data_type())?), diff --git a/datafusion/datasource/src/nested_schema_adapter/adapter.rs b/datafusion/datasource/src/nested_schema_adapter/adapter.rs index c9093f821d2e3..bf41304f29fb2 100644 --- a/datafusion/datasource/src/nested_schema_adapter/adapter.rs +++ b/datafusion/datasource/src/nested_schema_adapter/adapter.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -use super::mapping::adapt_column; +use datafusion_common::nested_struct::adapt_column; use crate::schema_adapter::{ create_field_mapping, SchemaAdapter, SchemaMapper, SchemaMapping, }; diff --git a/datafusion/datasource/src/nested_schema_adapter/mod.rs b/datafusion/datasource/src/nested_schema_adapter/mod.rs index 7e7c60590d34b..7494713337e05 100644 --- a/datafusion/datasource/src/nested_schema_adapter/mod.rs +++ b/datafusion/datasource/src/nested_schema_adapter/mod.rs @@ -19,7 +19,6 @@ pub mod adapter; pub mod factory; -mod mapping; #[cfg(test)] mod tests; From e3bcaf22596311c54a431365bc66bd72153ec9a7 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 12 Jun 2025 15:49:11 +0800 Subject: [PATCH 36/57] fix: reorder imports to maintain consistency in nested_struct module --- datafusion/common/src/nested_struct.rs | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/datafusion/common/src/nested_struct.rs b/datafusion/common/src/nested_struct.rs index bb2a93767a37e..b0a933eff1116 100644 --- a/datafusion/common/src/nested_struct.rs +++ b/datafusion/common/src/nested_struct.rs @@ -16,14 +16,12 @@ // under the License. use crate::error::Result; -use arrow::array::new_null_array; use arrow::{ - array::{Array, ArrayRef, StructArray}, + array::{new_null_array, Array, ArrayRef, StructArray}, compute::cast, datatypes::{DataType::Struct, Field}, }; use std::sync::Arc; - /// Adapt a struct column to match the target field type, handling nested structs recursively fn adapt_struct_column( source_col: &ArrayRef, From a4ff0ddead88587c621f7478d9073c557c779bd9 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 12 Jun 2025 16:09:57 +0800 Subject: [PATCH 37/57] fix cargo fmt errors --- datafusion/common/src/lib.rs | 2 +- datafusion/datasource/src/nested_schema_adapter/adapter.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/common/src/lib.rs b/datafusion/common/src/lib.rs index 19152b682ec3a..3b609d7722ecf 100644 --- a/datafusion/common/src/lib.rs +++ b/datafusion/common/src/lib.rs @@ -46,11 +46,11 @@ pub mod file_options; pub mod format; pub mod hash_utils; pub mod instant; +pub mod nested_struct; pub mod parsers; pub mod pruning; pub mod rounding; pub mod scalar; -pub mod nested_struct; pub mod spans; pub mod stats; pub mod test_util; diff --git a/datafusion/datasource/src/nested_schema_adapter/adapter.rs b/datafusion/datasource/src/nested_schema_adapter/adapter.rs index bf41304f29fb2..95af70b0c1f2d 100644 --- a/datafusion/datasource/src/nested_schema_adapter/adapter.rs +++ b/datafusion/datasource/src/nested_schema_adapter/adapter.rs @@ -15,7 +15,6 @@ // specific language governing permissions and limitations // under the License. -use datafusion_common::nested_struct::adapt_column; use crate::schema_adapter::{ create_field_mapping, SchemaAdapter, SchemaMapper, SchemaMapping, }; @@ -23,6 +22,7 @@ use arrow::{ array::ArrayRef, datatypes::{DataType::Struct, Field, Schema, SchemaRef}, }; +use datafusion_common::nested_struct::adapt_column; use datafusion_common::Result; use std::sync::Arc; From 5120d68f4f163172e6d1950553935c8927d406b6 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 12 Jun 2025 16:28:05 +0800 Subject: [PATCH 38/57] fix: add Default trait implementation for SchemaSource enum --- datafusion/core/src/datasource/listing/table.rs | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index e5810186c7ad9..ecf5c18cc1602 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -56,9 +56,10 @@ use object_store::ObjectStore; use std::{any::Any, collections::HashMap, str::FromStr, sync::Arc}; /// Indicates the source of the schema for a [`ListingTable`] // PartialEq required for assert_eq! in tests -#[derive(Debug, Clone, Copy, PartialEq)] +#[derive(Debug, Clone, Copy, PartialEq, Default)] pub enum SchemaSource { /// Schema is not yet set (initial state) + #[default] None, /// Schema was inferred from first table_path Inferred, @@ -88,12 +89,6 @@ pub struct ListingTableConfig { schema_adapter_factory: Option>, } -impl Default for SchemaSource { - fn default() -> Self { - SchemaSource::None - } -} - impl ListingTableConfig { /// Creates new [`ListingTableConfig`] for reading the specified URL pub fn new(table_path: ListingTableUrl) -> Self { From 2cb97b98c84f3fbfa0ce56646284cc6c2305215b Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 12 Jun 2025 16:37:48 +0800 Subject: [PATCH 39/57] docs: add comment indicating tests for adapt_column function --- datafusion/common/src/nested_struct.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/datafusion/common/src/nested_struct.rs b/datafusion/common/src/nested_struct.rs index b0a933eff1116..51ecd4e80d1b8 100644 --- a/datafusion/common/src/nested_struct.rs +++ b/datafusion/common/src/nested_struct.rs @@ -60,6 +60,8 @@ fn adapt_struct_column( } /// Adapt a column to match the target field type, handling nested structs specially +/// +// This is tested in nested_schema_adapter/tests.rs pub fn adapt_column(source_col: &ArrayRef, target_field: &Field) -> Result { match target_field.data_type() { Struct(target_fields) => adapt_struct_column(source_col, target_fields), From 5dd339e3fceb2069433d5be460450fcce0cfe622 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 12 Jun 2025 16:54:51 +0800 Subject: [PATCH 40/57] refactor(tests): reorganize and enhance schema evolution tests for nested fields --- .../src/nested_schema_adapter/tests.rs | 1153 ++++++++--------- 1 file changed, 574 insertions(+), 579 deletions(-) diff --git a/datafusion/datasource/src/nested_schema_adapter/tests.rs b/datafusion/datasource/src/nested_schema_adapter/tests.rs index 970402bb0b0e3..f3c1038827f5f 100644 --- a/datafusion/datasource/src/nested_schema_adapter/tests.rs +++ b/datafusion/datasource/src/nested_schema_adapter/tests.rs @@ -15,196 +15,555 @@ // specific language governing permissions and limitations // under the License. -mod tests { - use crate::nested_schema_adapter::{ +use crate::{ + nested_schema_adapter::{ NestedStructSchemaAdapter, NestedStructSchemaAdapterFactory, - }; - use crate::schema_adapter::{ - DefaultSchemaAdapterFactory, SchemaAdapter, SchemaAdapterFactory, - }; - use arrow::{ - array::{Array, StringBuilder, StructArray, TimestampMillisecondArray}, - compute::cast, - datatypes::{ - DataType::{Float64, Int16, Int32, Struct, Timestamp, Utf8}, - Field, Schema, SchemaRef, - TimeUnit::Millisecond, - }, - record_batch::RecordBatch, - }; - use datafusion_common::{ColumnStatistics, Result, ScalarValue}; - use std::sync::Arc; - - // ================================ - // Schema Creation Helper Functions - // ================================ - - /// Helper function to create a basic nested schema with additionalInfo - fn create_basic_nested_schema() -> SchemaRef { - Arc::new(Schema::new(vec![ - create_additional_info_field(false), // without reason field - ])) - } + }, + schema_adapter::{DefaultSchemaAdapterFactory, SchemaAdapter, SchemaAdapterFactory}, +}; +use arrow::{ + array::{Array, StringBuilder, StructArray, TimestampMillisecondArray}, + compute::cast, + datatypes::{ + DataType::{Float64, Int16, Int32, Struct, Timestamp, Utf8}, + Field, Schema, SchemaRef, + TimeUnit::Millisecond, + }, + record_batch::RecordBatch, +}; +use datafusion_common::{ColumnStatistics, Result, ScalarValue}; +use std::sync::Arc; +// ================================ +// Schema Creation Helper Functions +// ================================ + +/// Helper function to create a basic nested schema with additionalInfo +fn create_basic_nested_schema() -> SchemaRef { + Arc::new(Schema::new(vec![ + create_additional_info_field(false), // without reason field + ])) +} + +/// Helper function to create a deeply nested schema with additionalInfo including reason field +fn create_deep_nested_schema() -> SchemaRef { + Arc::new(Schema::new(vec![ + create_additional_info_field(true), // with reason field + ])) +} + +/// Helper function to create the additionalInfo field with or without the reason subfield +fn create_additional_info_field(with_reason: bool) -> Field { + let mut field_children = vec![ + Field::new("location", Utf8, true), + Field::new("timestamp_utc", Timestamp(Millisecond, None), true), + ]; - /// Helper function to create a deeply nested schema with additionalInfo including reason field - fn create_deep_nested_schema() -> SchemaRef { - Arc::new(Schema::new(vec![ - create_additional_info_field(true), // with reason field - ])) + // Add the reason field if requested (for target schema) + if with_reason { + field_children.push(create_reason_field()); } - /// Helper function to create the additionalInfo field with or without the reason subfield - fn create_additional_info_field(with_reason: bool) -> Field { - let mut field_children = vec![ - Field::new("location", Utf8, true), - Field::new("timestamp_utc", Timestamp(Millisecond, None), true), - ]; + Field::new("additionalInfo", Struct(field_children.into()), true) +} - // Add the reason field if requested (for target schema) - if with_reason { - field_children.push(create_reason_field()); - } +/// Helper function to create the reason nested field with its details subfield +fn create_reason_field() -> Field { + Field::new( + "reason", + Struct( + vec![ + Field::new("_level", Float64, true), + // Inline the details field creation + Field::new( + "details", + Struct( + vec![ + Field::new("rurl", Utf8, true), + Field::new("s", Float64, true), + Field::new("t", Utf8, true), + ] + .into(), + ), + true, + ), + ] + .into(), + ), + true, + ) +} - Field::new("additionalInfo", Struct(field_children.into()), true) - } +// ================================ +// Schema Evolution Tests +// ================================ + +#[test] +fn test_adapter_factory_selection() -> Result<()> { + // Test schemas for adapter selection logic + let simple_schema = Arc::new(Schema::new(vec![ + Field::new("id", Int32, false), + Field::new("name", Utf8, true), + Field::new("age", Int16, true), + ])); + + let nested_schema = Arc::new(Schema::new(vec![ + Field::new("id", Int32, false), + Field::new( + "metadata", + Struct( + vec![ + Field::new("created", Utf8, true), + Field::new("modified", Utf8, true), + ] + .into(), + ), + true, + ), + ])); - /// Helper function to create the reason nested field with its details subfield - fn create_reason_field() -> Field { + // Source schema with missing field + let source_schema = Arc::new(Schema::new(vec![ + Field::new("id", Int32, false), Field::new( - "reason", + "metadata", Struct( vec![ - Field::new("_level", Float64, true), - // Inline the details field creation - Field::new( - "details", - Struct( - vec![ - Field::new("rurl", Utf8, true), - Field::new("s", Float64, true), - Field::new("t", Utf8, true), - ] - .into(), - ), - true, - ), + Field::new("created", Utf8, true), + // "modified" field is missing ] .into(), ), true, - ) - } + ), + ])); + + // Test struct detection + assert!( + !NestedStructSchemaAdapterFactory::has_nested_structs(&simple_schema), + "Simple schema should not be detected as having nested structs" + ); + assert!( + NestedStructSchemaAdapterFactory::has_nested_structs(&nested_schema), + "Nested schema should be detected as having nested structs" + ); + + // Test adapter behavior with schema evolution + let default_adapter = + DefaultSchemaAdapterFactory.create(nested_schema.clone(), nested_schema.clone()); + let nested_adapter = NestedStructSchemaAdapterFactory + .create(nested_schema.clone(), nested_schema.clone()); + + // Default adapter should fail with schema evolution + assert!(default_adapter.map_schema(&source_schema).is_err()); + + // Nested adapter should handle schema evolution + assert!( + nested_adapter.map_schema(&source_schema).is_ok(), + "Nested adapter should handle schema with missing fields" + ); + + // Test factory selection logic + let adapter: Box = + if NestedStructSchemaAdapterFactory::has_nested_structs(&nested_schema) { + NestedStructSchemaAdapterFactory + .create(nested_schema.clone(), nested_schema.clone()) + } else { + DefaultSchemaAdapterFactory + .create(nested_schema.clone(), nested_schema.clone()) + }; + + assert!( + adapter.map_schema(&source_schema).is_ok(), + "Factory should select appropriate adapter that handles schema evolution" + ); + + Ok(()) +} + +#[test] +fn test_adapt_struct_with_added_nested_fields() -> Result<()> { + // Create test schemas + let (file_schema, table_schema) = create_test_schemas_with_nested_fields(); + + // Create batch with test data + let batch = create_test_batch_with_struct_data(&file_schema)?; + + // Create adapter and apply it + let mapped_batch = + adapt_batch_with_nested_schema_adapter(&file_schema, &table_schema, batch)?; + + // Verify the results + verify_adapted_batch_with_nested_fields(&mapped_batch, &table_schema)?; + + Ok(()) +} - // ================================ - // Schema Evolution Tests - // ================================ - - #[test] - fn test_adapter_factory_selection() -> Result<()> { - // Test schemas for adapter selection logic - let simple_schema = Arc::new(Schema::new(vec![ - Field::new("id", Int32, false), - Field::new("name", Utf8, true), - Field::new("age", Int16, true), - ])); - - let nested_schema = Arc::new(Schema::new(vec![ - Field::new("id", Int32, false), - Field::new( - "metadata", - Struct( - vec![ - Field::new("created", Utf8, true), - Field::new("modified", Utf8, true), - ] - .into(), +/// Create file and table schemas for testing nested field evolution +fn create_test_schemas_with_nested_fields() -> (SchemaRef, SchemaRef) { + // Create file schema with just location and timestamp_utc + let file_schema = Arc::new(Schema::new(vec![Field::new( + "info", + Struct( + vec![ + Field::new("location", Utf8, true), + Field::new( + "timestamp_utc", + Timestamp(Millisecond, Some("UTC".into())), + true, ), - true, - ), - ])); - - // Source schema with missing field - let source_schema = Arc::new(Schema::new(vec![ - Field::new("id", Int32, false), - Field::new( - "metadata", - Struct( - vec![ - Field::new("created", Utf8, true), - // "modified" field is missing - ] - .into(), + ] + .into(), + ), + true, + )])); + + // Create table schema with additional nested reason field + let table_schema = Arc::new(Schema::new(vec![Field::new( + "info", + Struct( + vec![ + Field::new("location", Utf8, true), + Field::new( + "timestamp_utc", + Timestamp(Millisecond, Some("UTC".into())), + true, ), - true, - ), - ])); + Field::new( + "reason", + Struct( + vec![ + Field::new("_level", Float64, true), + Field::new( + "details", + Struct( + vec![ + Field::new("rurl", Utf8, true), + Field::new("s", Float64, true), + Field::new("t", Utf8, true), + ] + .into(), + ), + true, + ), + ] + .into(), + ), + true, + ), + ] + .into(), + ), + true, + )])); - // Test struct detection - assert!( - !NestedStructSchemaAdapterFactory::has_nested_structs(&simple_schema), - "Simple schema should not be detected as having nested structs" - ); - assert!( - NestedStructSchemaAdapterFactory::has_nested_structs(&nested_schema), - "Nested schema should be detected as having nested structs" + (file_schema, table_schema) +} + +/// Create a test RecordBatch with struct data matching the file schema +fn create_test_batch_with_struct_data(file_schema: &SchemaRef) -> Result { + let mut location_builder = StringBuilder::new(); + location_builder.append_value("San Francisco"); + location_builder.append_value("New York"); + + // Create timestamp array + let timestamp_array = TimestampMillisecondArray::from(vec![ + Some(1640995200000), // 2022-01-01 + Some(1641081600000), // 2022-01-02 + ]); + + // Create data type with UTC timezone to match the schema + let timestamp_type = Timestamp(Millisecond, Some("UTC".into())); + + // Cast the timestamp array to include the timezone metadata + let timestamp_array = cast(×tamp_array, ×tamp_type)?; + + let info_struct = StructArray::from(vec![ + ( + Arc::new(Field::new("location", Utf8, true)), + Arc::new(location_builder.finish()) as Arc, + ), + ( + Arc::new(Field::new("timestamp_utc", timestamp_type, true)), + timestamp_array, + ), + ]); + + Ok(RecordBatch::try_new( + Arc::clone(file_schema), + vec![Arc::new(info_struct)], + )?) +} + +/// Apply the nested schema adapter to the batch +fn adapt_batch_with_nested_schema_adapter( + file_schema: &SchemaRef, + table_schema: &SchemaRef, + batch: RecordBatch, +) -> Result { + let adapter = NestedStructSchemaAdapter::new( + Arc::clone(table_schema), + Arc::clone(table_schema), + ); + + let (mapper, _) = adapter.map_schema(file_schema.as_ref())?; + mapper.map_batch(batch) +} + +/// Verify the adapted batch has the expected structure and data +fn verify_adapted_batch_with_nested_fields( + mapped_batch: &RecordBatch, + table_schema: &SchemaRef, +) -> Result<()> { + // Verify the mapped batch structure and data + assert_eq!(mapped_batch.schema(), *table_schema); + assert_eq!(mapped_batch.num_rows(), 2); + + // Extract and verify the info struct column + let info_col = mapped_batch.column(0); + let info_array = info_col + .as_any() + .downcast_ref::() + .expect("Expected info column to be a StructArray"); + + // Verify the original fields are preserved + verify_preserved_fields(info_array)?; + + // Verify the reason field exists with correct structure + verify_reason_field_structure(info_array)?; + + Ok(()) +} + +/// Verify the original fields from file schema are preserved in the adapted batch +fn verify_preserved_fields(info_array: &StructArray) -> Result<()> { + // Verify location field + let location_col = info_array + .column_by_name("location") + .expect("Expected location field in struct"); + let location_array = location_col + .as_any() + .downcast_ref::() + .expect("Expected location to be a StringArray"); + + // Verify the location values are preserved + assert_eq!(location_array.value(0), "San Francisco"); + assert_eq!(location_array.value(1), "New York"); + + // Verify timestamp field + let timestamp_col = info_array + .column_by_name("timestamp_utc") + .expect("Expected timestamp_utc field in struct"); + let timestamp_array = timestamp_col + .as_any() + .downcast_ref::() + .expect("Expected timestamp_utc to be a TimestampMillisecondArray"); + + assert_eq!(timestamp_array.value(0), 1640995200000); + assert_eq!(timestamp_array.value(1), 1641081600000); + + Ok(()) +} + +/// Verify the added reason field structure and null values +fn verify_reason_field_structure(info_array: &StructArray) -> Result<()> { + // Verify the reason field exists and is null + let reason_col = info_array + .column_by_name("reason") + .expect("Expected reason field in struct"); + let reason_array = reason_col + .as_any() + .downcast_ref::() + .expect("Expected reason to be a StructArray"); + + // Verify reason has correct structure + assert_eq!(reason_array.fields().len(), 2); + assert!(reason_array.column_by_name("_level").is_some()); + assert!(reason_array.column_by_name("details").is_some()); + + // Verify details field has correct nested structure + let details_col = reason_array + .column_by_name("details") + .expect("Expected details field in reason struct"); + let details_array = details_col + .as_any() + .downcast_ref::() + .expect("Expected details to be a StructArray"); + + assert_eq!(details_array.fields().len(), 3); + assert!(details_array.column_by_name("rurl").is_some()); + assert!(details_array.column_by_name("s").is_some()); + assert!(details_array.column_by_name("t").is_some()); + + // Verify all added fields are null + for i in 0..2 { + assert!(reason_array.is_null(i), "reason field should be null"); + } + + Ok(()) +} + +// ================================ +// Data Mapping Tests +// ================================ + +// Helper function to verify column statistics match expected values +fn verify_column_statistics( + stats: &ColumnStatistics, + expected_null_count: Option, + expected_distinct_count: Option, + expected_min: Option, + expected_max: Option, + expected_sum: Option, +) { + if let Some(count) = expected_null_count { + assert_eq!( + stats.null_count, + datafusion_common::stats::Precision::Exact(count), + "Null count should match expected value" ); + } - // Test adapter behavior with schema evolution - let default_adapter = DefaultSchemaAdapterFactory - .create(nested_schema.clone(), nested_schema.clone()); - let nested_adapter = NestedStructSchemaAdapterFactory - .create(nested_schema.clone(), nested_schema.clone()); + if let Some(count) = expected_distinct_count { + assert_eq!( + stats.distinct_count, + datafusion_common::stats::Precision::Exact(count), + "Distinct count should match expected value" + ); + } - // Default adapter should fail with schema evolution - assert!(default_adapter.map_schema(&source_schema).is_err()); + if let Some(min) = expected_min { + assert_eq!( + stats.min_value, + datafusion_common::stats::Precision::Exact(min), + "Min value should match expected value" + ); + } - // Nested adapter should handle schema evolution - assert!( - nested_adapter.map_schema(&source_schema).is_ok(), - "Nested adapter should handle schema with missing fields" + if let Some(max) = expected_max { + assert_eq!( + stats.max_value, + datafusion_common::stats::Precision::Exact(max), + "Max value should match expected value" ); + } - // Test factory selection logic - let adapter: Box = - if NestedStructSchemaAdapterFactory::has_nested_structs(&nested_schema) { - NestedStructSchemaAdapterFactory - .create(nested_schema.clone(), nested_schema.clone()) - } else { - DefaultSchemaAdapterFactory - .create(nested_schema.clone(), nested_schema.clone()) - }; - - assert!( - adapter.map_schema(&source_schema).is_ok(), - "Factory should select appropriate adapter that handles schema evolution" + if let Some(sum) = expected_sum { + assert_eq!( + stats.sum_value, + datafusion_common::stats::Precision::Exact(sum), + "Sum value should match expected value" ); + } +} - Ok(()) +// Helper to create test column statistics +fn create_test_column_statistics( + null_count: usize, + distinct_count: usize, + min_value: Option, + max_value: Option, + sum_value: Option, +) -> ColumnStatistics { + ColumnStatistics { + null_count: datafusion_common::stats::Precision::Exact(null_count), + distinct_count: datafusion_common::stats::Precision::Exact(distinct_count), + min_value: min_value.map_or_else( + || datafusion_common::stats::Precision::Absent, + datafusion_common::stats::Precision::Exact, + ), + max_value: max_value.map_or_else( + || datafusion_common::stats::Precision::Absent, + datafusion_common::stats::Precision::Exact, + ), + sum_value: sum_value.map_or_else( + || datafusion_common::stats::Precision::Absent, + datafusion_common::stats::Precision::Exact, + ), } +} - #[test] - fn test_adapt_struct_with_added_nested_fields() -> Result<()> { - // Create test schemas - let (file_schema, table_schema) = create_test_schemas_with_nested_fields(); +#[test] +fn test_map_column_statistics_basic() -> Result<()> { + // Test statistics mapping with a simple schema + let file_schema = create_basic_nested_schema(); + let table_schema = create_deep_nested_schema(); + + let adapter = NestedStructSchemaAdapter::new( + Arc::clone(&table_schema), + Arc::clone(&table_schema), + ); + + let (mapper, _) = adapter.map_schema(file_schema.as_ref())?; + + // Create test statistics for additionalInfo column + let file_stats = vec![create_test_column_statistics( + 5, + 100, + Some(ScalarValue::Utf8(Some("min_value".to_string()))), + Some(ScalarValue::Utf8(Some("max_value".to_string()))), + Some(ScalarValue::Utf8(Some("sum_value".to_string()))), + )]; + + // Map statistics + let table_stats = mapper.map_column_statistics(&file_stats)?; + + // Verify count and content + assert_eq!( + table_stats.len(), + 1, + "Should have stats for one struct column" + ); + verify_column_statistics( + &table_stats[0], + Some(5), + Some(100), + Some(ScalarValue::Utf8(Some("min_value".to_string()))), + Some(ScalarValue::Utf8(Some("max_value".to_string()))), + Some(ScalarValue::Utf8(Some("sum_value".to_string()))), + ); + + Ok(()) +} - // Create batch with test data - let batch = create_test_batch_with_struct_data(&file_schema)?; +#[test] +fn test_map_column_statistics_empty() -> Result<()> { + // Test statistics mapping with empty input + let file_schema = create_basic_nested_schema(); + let table_schema = create_deep_nested_schema(); - // Create adapter and apply it - let mapped_batch = - adapt_batch_with_nested_schema_adapter(&file_schema, &table_schema, batch)?; + let adapter = NestedStructSchemaAdapter::new( + Arc::clone(&table_schema), + Arc::clone(&table_schema), + ); - // Verify the results - verify_adapted_batch_with_nested_fields(&mapped_batch, &table_schema)?; + let (mapper, _) = adapter.map_schema(file_schema.as_ref())?; - Ok(()) - } + // Test with missing statistics + let empty_stats = vec![]; + let mapped_empty_stats = mapper.map_column_statistics(&empty_stats)?; + + assert_eq!( + mapped_empty_stats.len(), + 1, + "Should have stats for one column even with empty input" + ); + + assert_eq!( + mapped_empty_stats[0], + ColumnStatistics::new_unknown(), + "Empty input should result in unknown statistics" + ); + + Ok(()) +} - /// Create file and table schemas for testing nested field evolution - fn create_test_schemas_with_nested_fields() -> (SchemaRef, SchemaRef) { - // Create file schema with just location and timestamp_utc - let file_schema = Arc::new(Schema::new(vec![Field::new( - "info", +#[test] +fn test_map_column_statistics_multiple_columns() -> Result<()> { + // Create schemas with multiple columns + let file_schema = Arc::new(Schema::new(vec![ + Field::new("id", Int32, false), + Field::new( + "additionalInfo", Struct( vec![ Field::new("location", Utf8, true), @@ -217,11 +576,13 @@ mod tests { .into(), ), true, - )])); + ), + ])); - // Create table schema with additional nested reason field - let table_schema = Arc::new(Schema::new(vec![Field::new( - "info", + let table_schema = Arc::new(Schema::new(vec![ + Field::new("id", Int32, false), + Field::new( + "additionalInfo", Struct( vec![ Field::new("location", Utf8, true), @@ -232,432 +593,66 @@ mod tests { ), Field::new( "reason", - Struct( - vec![ - Field::new("_level", Float64, true), - Field::new( - "details", - Struct( - vec![ - Field::new("rurl", Utf8, true), - Field::new("s", Float64, true), - Field::new("t", Utf8, true), - ] - .into(), - ), - true, - ), - ] - .into(), - ), + Struct(vec![Field::new("_level", Float64, true)].into()), true, ), ] .into(), ), true, - )])); - - (file_schema, table_schema) - } - - /// Create a test RecordBatch with struct data matching the file schema - fn create_test_batch_with_struct_data( - file_schema: &SchemaRef, - ) -> Result { - let mut location_builder = StringBuilder::new(); - location_builder.append_value("San Francisco"); - location_builder.append_value("New York"); - - // Create timestamp array - let timestamp_array = TimestampMillisecondArray::from(vec![ - Some(1640995200000), // 2022-01-01 - Some(1641081600000), // 2022-01-02 - ]); - - // Create data type with UTC timezone to match the schema - let timestamp_type = Timestamp(Millisecond, Some("UTC".into())); - - // Cast the timestamp array to include the timezone metadata - let timestamp_array = cast(×tamp_array, ×tamp_type)?; - - let info_struct = StructArray::from(vec![ - ( - Arc::new(Field::new("location", Utf8, true)), - Arc::new(location_builder.finish()) as Arc, - ), - ( - Arc::new(Field::new("timestamp_utc", timestamp_type, true)), - timestamp_array, - ), - ]); - - Ok(RecordBatch::try_new( - Arc::clone(file_schema), - vec![Arc::new(info_struct)], - )?) - } - - /// Apply the nested schema adapter to the batch - fn adapt_batch_with_nested_schema_adapter( - file_schema: &SchemaRef, - table_schema: &SchemaRef, - batch: RecordBatch, - ) -> Result { - let adapter = NestedStructSchemaAdapter::new( - Arc::clone(table_schema), - Arc::clone(table_schema), - ); - - let (mapper, _) = adapter.map_schema(file_schema.as_ref())?; - mapper.map_batch(batch) - } - - /// Verify the adapted batch has the expected structure and data - fn verify_adapted_batch_with_nested_fields( - mapped_batch: &RecordBatch, - table_schema: &SchemaRef, - ) -> Result<()> { - // Verify the mapped batch structure and data - assert_eq!(mapped_batch.schema(), *table_schema); - assert_eq!(mapped_batch.num_rows(), 2); - - // Extract and verify the info struct column - let info_col = mapped_batch.column(0); - let info_array = info_col - .as_any() - .downcast_ref::() - .expect("Expected info column to be a StructArray"); - - // Verify the original fields are preserved - verify_preserved_fields(info_array)?; - - // Verify the reason field exists with correct structure - verify_reason_field_structure(info_array)?; - - Ok(()) - } - - /// Verify the original fields from file schema are preserved in the adapted batch - fn verify_preserved_fields(info_array: &StructArray) -> Result<()> { - // Verify location field - let location_col = info_array - .column_by_name("location") - .expect("Expected location field in struct"); - let location_array = location_col - .as_any() - .downcast_ref::() - .expect("Expected location to be a StringArray"); - - // Verify the location values are preserved - assert_eq!(location_array.value(0), "San Francisco"); - assert_eq!(location_array.value(1), "New York"); - - // Verify timestamp field - let timestamp_col = info_array - .column_by_name("timestamp_utc") - .expect("Expected timestamp_utc field in struct"); - let timestamp_array = timestamp_col - .as_any() - .downcast_ref::() - .expect("Expected timestamp_utc to be a TimestampMillisecondArray"); - - assert_eq!(timestamp_array.value(0), 1640995200000); - assert_eq!(timestamp_array.value(1), 1641081600000); - - Ok(()) - } - - /// Verify the added reason field structure and null values - fn verify_reason_field_structure(info_array: &StructArray) -> Result<()> { - // Verify the reason field exists and is null - let reason_col = info_array - .column_by_name("reason") - .expect("Expected reason field in struct"); - let reason_array = reason_col - .as_any() - .downcast_ref::() - .expect("Expected reason to be a StructArray"); - - // Verify reason has correct structure - assert_eq!(reason_array.fields().len(), 2); - assert!(reason_array.column_by_name("_level").is_some()); - assert!(reason_array.column_by_name("details").is_some()); - - // Verify details field has correct nested structure - let details_col = reason_array - .column_by_name("details") - .expect("Expected details field in reason struct"); - let details_array = details_col - .as_any() - .downcast_ref::() - .expect("Expected details to be a StructArray"); - - assert_eq!(details_array.fields().len(), 3); - assert!(details_array.column_by_name("rurl").is_some()); - assert!(details_array.column_by_name("s").is_some()); - assert!(details_array.column_by_name("t").is_some()); - - // Verify all added fields are null - for i in 0..2 { - assert!(reason_array.is_null(i), "reason field should be null"); - } - - Ok(()) - } - - // ================================ - // Data Mapping Tests - // ================================ - - // Helper function to verify column statistics match expected values - fn verify_column_statistics( - stats: &ColumnStatistics, - expected_null_count: Option, - expected_distinct_count: Option, - expected_min: Option, - expected_max: Option, - expected_sum: Option, - ) { - if let Some(count) = expected_null_count { - assert_eq!( - stats.null_count, - datafusion_common::stats::Precision::Exact(count), - "Null count should match expected value" - ); - } - - if let Some(count) = expected_distinct_count { - assert_eq!( - stats.distinct_count, - datafusion_common::stats::Precision::Exact(count), - "Distinct count should match expected value" - ); - } - - if let Some(min) = expected_min { - assert_eq!( - stats.min_value, - datafusion_common::stats::Precision::Exact(min), - "Min value should match expected value" - ); - } - - if let Some(max) = expected_max { - assert_eq!( - stats.max_value, - datafusion_common::stats::Precision::Exact(max), - "Max value should match expected value" - ); - } - - if let Some(sum) = expected_sum { - assert_eq!( - stats.sum_value, - datafusion_common::stats::Precision::Exact(sum), - "Sum value should match expected value" - ); - } - } - - // Helper to create test column statistics - fn create_test_column_statistics( - null_count: usize, - distinct_count: usize, - min_value: Option, - max_value: Option, - sum_value: Option, - ) -> ColumnStatistics { - ColumnStatistics { - null_count: datafusion_common::stats::Precision::Exact(null_count), - distinct_count: datafusion_common::stats::Precision::Exact(distinct_count), - min_value: min_value.map_or_else( - || datafusion_common::stats::Precision::Absent, - datafusion_common::stats::Precision::Exact, - ), - max_value: max_value.map_or_else( - || datafusion_common::stats::Precision::Absent, - datafusion_common::stats::Precision::Exact, - ), - sum_value: sum_value.map_or_else( - || datafusion_common::stats::Precision::Absent, - datafusion_common::stats::Precision::Exact, - ), - } - } - - #[test] - fn test_map_column_statistics_basic() -> Result<()> { - // Test statistics mapping with a simple schema - let file_schema = create_basic_nested_schema(); - let table_schema = create_deep_nested_schema(); - - let adapter = NestedStructSchemaAdapter::new( - Arc::clone(&table_schema), - Arc::clone(&table_schema), - ); - - let (mapper, _) = adapter.map_schema(file_schema.as_ref())?; - - // Create test statistics for additionalInfo column - let file_stats = vec![create_test_column_statistics( - 5, + ), + Field::new("status", Utf8, true), // Extra column in table schema + ])); + + // Create adapter and mapping + let adapter = NestedStructSchemaAdapter::new( + Arc::clone(&table_schema), + Arc::clone(&table_schema), + ); + + let (mapper, _) = adapter.map_schema(file_schema.as_ref())?; + + // Create file column statistics + let file_stats = vec![ + create_test_column_statistics( + 0, 100, - Some(ScalarValue::Utf8(Some("min_value".to_string()))), - Some(ScalarValue::Utf8(Some("max_value".to_string()))), - Some(ScalarValue::Utf8(Some("sum_value".to_string()))), - )]; - - // Map statistics - let table_stats = mapper.map_column_statistics(&file_stats)?; - - // Verify count and content - assert_eq!( - table_stats.len(), - 1, - "Should have stats for one struct column" - ); - verify_column_statistics( - &table_stats[0], - Some(5), - Some(100), - Some(ScalarValue::Utf8(Some("min_value".to_string()))), - Some(ScalarValue::Utf8(Some("max_value".to_string()))), - Some(ScalarValue::Utf8(Some("sum_value".to_string()))), - ); - - Ok(()) - } - - #[test] - fn test_map_column_statistics_empty() -> Result<()> { - // Test statistics mapping with empty input - let file_schema = create_basic_nested_schema(); - let table_schema = create_deep_nested_schema(); - - let adapter = NestedStructSchemaAdapter::new( - Arc::clone(&table_schema), - Arc::clone(&table_schema), - ); - - let (mapper, _) = adapter.map_schema(file_schema.as_ref())?; - - // Test with missing statistics - let empty_stats = vec![]; - let mapped_empty_stats = mapper.map_column_statistics(&empty_stats)?; - - assert_eq!( - mapped_empty_stats.len(), - 1, - "Should have stats for one column even with empty input" - ); - - assert_eq!( - mapped_empty_stats[0], - ColumnStatistics::new_unknown(), - "Empty input should result in unknown statistics" - ); - - Ok(()) - } - - #[test] - fn test_map_column_statistics_multiple_columns() -> Result<()> { - // Create schemas with multiple columns - let file_schema = Arc::new(Schema::new(vec![ - Field::new("id", Int32, false), - Field::new( - "additionalInfo", - Struct( - vec![ - Field::new("location", Utf8, true), - Field::new( - "timestamp_utc", - Timestamp(Millisecond, Some("UTC".into())), - true, - ), - ] - .into(), - ), - true, - ), - ])); - - let table_schema = Arc::new(Schema::new(vec![ - Field::new("id", Int32, false), - Field::new( - "additionalInfo", - Struct( - vec![ - Field::new("location", Utf8, true), - Field::new( - "timestamp_utc", - Timestamp(Millisecond, Some("UTC".into())), - true, - ), - Field::new( - "reason", - Struct(vec![Field::new("_level", Float64, true)].into()), - true, - ), - ] - .into(), - ), - true, - ), - Field::new("status", Utf8, true), // Extra column in table schema - ])); - - // Create adapter and mapping - let adapter = NestedStructSchemaAdapter::new( - Arc::clone(&table_schema), - Arc::clone(&table_schema), - ); - - let (mapper, _) = adapter.map_schema(file_schema.as_ref())?; - - // Create file column statistics - let file_stats = vec![ - create_test_column_statistics( - 0, - 100, - Some(ScalarValue::Int32(Some(1))), - Some(ScalarValue::Int32(Some(100))), - Some(ScalarValue::Int32(Some(5100))), - ), - create_test_column_statistics(10, 50, None, None, None), - ]; - - // Map statistics - let table_stats = mapper.map_column_statistics(&file_stats)?; - - // Verify mapped statistics - assert_eq!( - table_stats.len(), - 3, - "Should have stats for all 3 columns in table schema" - ); - - // Verify ID column stats - verify_column_statistics( - &table_stats[0], - Some(0), - Some(100), Some(ScalarValue::Int32(Some(1))), Some(ScalarValue::Int32(Some(100))), Some(ScalarValue::Int32(Some(5100))), - ); - - // Verify additionalInfo column stats - verify_column_statistics(&table_stats[1], Some(10), Some(50), None, None, None); - - // Verify status column has unknown stats - assert_eq!( - table_stats[2], - ColumnStatistics::new_unknown(), - "Missing column should have unknown statistics" - ); - - Ok(()) - } + ), + create_test_column_statistics(10, 50, None, None, None), + ]; + + // Map statistics + let table_stats = mapper.map_column_statistics(&file_stats)?; + + // Verify mapped statistics + assert_eq!( + table_stats.len(), + 3, + "Should have stats for all 3 columns in table schema" + ); + + // Verify ID column stats + verify_column_statistics( + &table_stats[0], + Some(0), + Some(100), + Some(ScalarValue::Int32(Some(1))), + Some(ScalarValue::Int32(Some(100))), + Some(ScalarValue::Int32(Some(5100))), + ); + + // Verify additionalInfo column stats + verify_column_statistics(&table_stats[1], Some(10), Some(50), None, None, None); + + // Verify status column has unknown stats + assert_eq!( + table_stats[2], + ColumnStatistics::new_unknown(), + "Missing column should have unknown statistics" + ); + + Ok(()) } From 8809c3da34130a03f3fa2730eda17c1d98c5cc58 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 12 Jun 2025 17:37:52 +0800 Subject: [PATCH 41/57] reorganize tests --- .../src/nested_schema_adapter/tests.rs | 88 +++---------------- 1 file changed, 11 insertions(+), 77 deletions(-) diff --git a/datafusion/datasource/src/nested_schema_adapter/tests.rs b/datafusion/datasource/src/nested_schema_adapter/tests.rs index f3c1038827f5f..d9e8aacd8da50 100644 --- a/datafusion/datasource/src/nested_schema_adapter/tests.rs +++ b/datafusion/datasource/src/nested_schema_adapter/tests.rs @@ -483,82 +483,7 @@ fn create_test_column_statistics( } #[test] -fn test_map_column_statistics_basic() -> Result<()> { - // Test statistics mapping with a simple schema - let file_schema = create_basic_nested_schema(); - let table_schema = create_deep_nested_schema(); - - let adapter = NestedStructSchemaAdapter::new( - Arc::clone(&table_schema), - Arc::clone(&table_schema), - ); - - let (mapper, _) = adapter.map_schema(file_schema.as_ref())?; - - // Create test statistics for additionalInfo column - let file_stats = vec![create_test_column_statistics( - 5, - 100, - Some(ScalarValue::Utf8(Some("min_value".to_string()))), - Some(ScalarValue::Utf8(Some("max_value".to_string()))), - Some(ScalarValue::Utf8(Some("sum_value".to_string()))), - )]; - - // Map statistics - let table_stats = mapper.map_column_statistics(&file_stats)?; - - // Verify count and content - assert_eq!( - table_stats.len(), - 1, - "Should have stats for one struct column" - ); - verify_column_statistics( - &table_stats[0], - Some(5), - Some(100), - Some(ScalarValue::Utf8(Some("min_value".to_string()))), - Some(ScalarValue::Utf8(Some("max_value".to_string()))), - Some(ScalarValue::Utf8(Some("sum_value".to_string()))), - ); - - Ok(()) -} - -#[test] -fn test_map_column_statistics_empty() -> Result<()> { - // Test statistics mapping with empty input - let file_schema = create_basic_nested_schema(); - let table_schema = create_deep_nested_schema(); - - let adapter = NestedStructSchemaAdapter::new( - Arc::clone(&table_schema), - Arc::clone(&table_schema), - ); - - let (mapper, _) = adapter.map_schema(file_schema.as_ref())?; - - // Test with missing statistics - let empty_stats = vec![]; - let mapped_empty_stats = mapper.map_column_statistics(&empty_stats)?; - - assert_eq!( - mapped_empty_stats.len(), - 1, - "Should have stats for one column even with empty input" - ); - - assert_eq!( - mapped_empty_stats[0], - ColumnStatistics::new_unknown(), - "Empty input should result in unknown statistics" - ); - - Ok(()) -} - -#[test] -fn test_map_column_statistics_multiple_columns() -> Result<()> { +fn test_map_column_statistics() -> Result<()> { // Create schemas with multiple columns let file_schema = Arc::new(Schema::new(vec![ Field::new("id", Int32, false), @@ -651,8 +576,17 @@ fn test_map_column_statistics_multiple_columns() -> Result<()> { assert_eq!( table_stats[2], ColumnStatistics::new_unknown(), - "Missing column should have unknown statistics" + "Missing column should have unknown statistics", ); + // Verify behavior when no statistics are provided + let empty_stats = vec![]; + let missing_stats = mapper.map_column_statistics(&empty_stats)?; + + assert_eq!(missing_stats.len(), 3); + for stat in missing_stats { + assert_eq!(stat, ColumnStatistics::new_unknown()); + } + Ok(()) } From d08e058c298bc7f3e697baa4757fde41607f8141 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 12 Jun 2025 18:14:18 +0800 Subject: [PATCH 42/57] test: add unit tests for adapt_column function handling nested structs --- datafusion/common/src/nested_struct.rs | 84 ++++++++++++++++++++++++++ 1 file changed, 84 insertions(+) diff --git a/datafusion/common/src/nested_struct.rs b/datafusion/common/src/nested_struct.rs index 51ecd4e80d1b8..3ebc8c6ab15fd 100644 --- a/datafusion/common/src/nested_struct.rs +++ b/datafusion/common/src/nested_struct.rs @@ -68,3 +68,87 @@ pub fn adapt_column(source_col: &ArrayRef, target_field: &Field) -> Result Ok(cast(source_col, target_field.data_type())?), } } + +#[cfg(test)] +mod tests { + use super::*; + use arrow::array::{Int32Array, Int64Array, StringArray}; + use arrow::datatypes::{DataType, Field}; + + #[test] + fn test_adapt_simple_column() { + let source = Arc::new(Int32Array::from(vec![1, 2, 3])) as ArrayRef; + let target_field = Field::new("ints", DataType::Int64, true); + let result = adapt_column(&source, &target_field).unwrap(); + let result = result.as_any().downcast_ref::().unwrap(); + assert_eq!(result.len(), 3); + assert_eq!(result.value(0), 1); + assert_eq!(result.value(1), 2); + assert_eq!(result.value(2), 3); + } + + #[test] + fn test_adapt_struct_with_missing_field() { + let a_array = Arc::new(Int32Array::from(vec![1, 2])) as ArrayRef; + let source_struct = StructArray::from(vec![( + Arc::new(Field::new("a", DataType::Int32, true)), + a_array.clone(), + )]); + let source_col = Arc::new(source_struct) as ArrayRef; + + let target_field = Field::new( + "s", + Struct( + vec![ + Arc::new(Field::new("a", DataType::Int32, true)), + Arc::new(Field::new("b", DataType::Utf8, true)), + ] + .into(), + ), + true, + ); + + let result = adapt_column(&source_col, &target_field).unwrap(); + let struct_array = result.as_any().downcast_ref::().unwrap(); + assert_eq!(struct_array.fields().len(), 2); + let a_result = struct_array + .column_by_name("a") + .unwrap() + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(a_result.value(0), 1); + assert_eq!(a_result.value(1), 2); + + let b_result = struct_array + .column_by_name("b") + .unwrap() + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(b_result.len(), 2); + assert!(b_result.is_null(0)); + assert!(b_result.is_null(1)); + } + + #[test] + fn test_adapt_struct_source_not_struct() { + let source = Arc::new(Int32Array::from(vec![10, 20])) as ArrayRef; + let target_field = Field::new( + "s", + Struct(vec![Arc::new(Field::new("a", DataType::Int32, true))].into()), + true, + ); + + let result = adapt_column(&source, &target_field).unwrap(); + let struct_array = result.as_any().downcast_ref::().unwrap(); + assert_eq!(struct_array.len(), 2); + let a_result = struct_array + .column_by_name("a") + .unwrap() + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(a_result.null_count(), 2); + } +} From 68ad4fceeff63bca0076397f6b53e939b227a192 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 12 Jun 2025 18:18:29 +0800 Subject: [PATCH 43/57] refactor(tests): introduce helper function for downcasting columns in StructArray --- datafusion/common/src/nested_struct.rs | 36 ++++++++++++-------------- 1 file changed, 16 insertions(+), 20 deletions(-) diff --git a/datafusion/common/src/nested_struct.rs b/datafusion/common/src/nested_struct.rs index 3ebc8c6ab15fd..4a24db554cbcf 100644 --- a/datafusion/common/src/nested_struct.rs +++ b/datafusion/common/src/nested_struct.rs @@ -60,8 +60,6 @@ fn adapt_struct_column( } /// Adapt a column to match the target field type, handling nested structs specially -/// -// This is tested in nested_schema_adapter/tests.rs pub fn adapt_column(source_col: &ArrayRef, target_field: &Field) -> Result { match target_field.data_type() { Struct(target_fields) => adapt_struct_column(source_col, target_fields), @@ -75,6 +73,19 @@ mod tests { use arrow::array::{Int32Array, Int64Array, StringArray}; use arrow::datatypes::{DataType, Field}; + /// Helper function to extract and downcast a column from a StructArray + fn get_column_as(struct_array: &StructArray, column_name: &str) -> &T + where + T: Array, + { + struct_array + .column_by_name(column_name) + .unwrap() + .as_any() + .downcast_ref::() + .unwrap() + } + #[test] fn test_adapt_simple_column() { let source = Arc::new(Int32Array::from(vec![1, 2, 3])) as ArrayRef; @@ -111,21 +122,11 @@ mod tests { let result = adapt_column(&source_col, &target_field).unwrap(); let struct_array = result.as_any().downcast_ref::().unwrap(); assert_eq!(struct_array.fields().len(), 2); - let a_result = struct_array - .column_by_name("a") - .unwrap() - .as_any() - .downcast_ref::() - .unwrap(); + let a_result = get_column_as::(&struct_array, "a"); assert_eq!(a_result.value(0), 1); assert_eq!(a_result.value(1), 2); - let b_result = struct_array - .column_by_name("b") - .unwrap() - .as_any() - .downcast_ref::() - .unwrap(); + let b_result = get_column_as::(&struct_array, "b"); assert_eq!(b_result.len(), 2); assert!(b_result.is_null(0)); assert!(b_result.is_null(1)); @@ -143,12 +144,7 @@ mod tests { let result = adapt_column(&source, &target_field).unwrap(); let struct_array = result.as_any().downcast_ref::().unwrap(); assert_eq!(struct_array.len(), 2); - let a_result = struct_array - .column_by_name("a") - .unwrap() - .as_any() - .downcast_ref::() - .unwrap(); + let a_result = get_column_as::(&struct_array, "a"); assert_eq!(a_result.null_count(), 2); } } From 9d1daa29657018b82e33ac6df34a6d9d1d3c6976 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 12 Jun 2025 18:26:49 +0800 Subject: [PATCH 44/57] refactor(tests): reorganize imports, simplify get_column_as lifetime --- datafusion/common/src/nested_struct.rs | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/datafusion/common/src/nested_struct.rs b/datafusion/common/src/nested_struct.rs index 4a24db554cbcf..a817e70bfd8ef 100644 --- a/datafusion/common/src/nested_struct.rs +++ b/datafusion/common/src/nested_struct.rs @@ -70,9 +70,10 @@ pub fn adapt_column(source_col: &ArrayRef, target_field: &Field) -> Result(struct_array: &StructArray, column_name: &str) -> &T where From e9743488f5d8bdd40cf1fe1aec3e8253c8d2b8cf Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 12 Jun 2025 18:31:40 +0800 Subject: [PATCH 45/57] refactor(tests): replace get_column_as function with macro for downcasting columns in StructArray --- datafusion/common/src/nested_struct.rs | 27 +++++++++++++------------- 1 file changed, 13 insertions(+), 14 deletions(-) diff --git a/datafusion/common/src/nested_struct.rs b/datafusion/common/src/nested_struct.rs index a817e70bfd8ef..f962b399d8230 100644 --- a/datafusion/common/src/nested_struct.rs +++ b/datafusion/common/src/nested_struct.rs @@ -74,17 +74,16 @@ mod tests { array::{Int32Array, Int64Array, StringArray}, datatypes::{DataType, Field}, }; - /// Helper function to extract and downcast a column from a StructArray - fn get_column_as(struct_array: &StructArray, column_name: &str) -> &T - where - T: Array, - { - struct_array - .column_by_name(column_name) - .unwrap() - .as_any() - .downcast_ref::() - .unwrap() + /// Macro to extract and downcast a column from a StructArray + macro_rules! get_column_as { + ($struct_array:expr, $column_name:expr, $array_type:ty) => { + $struct_array + .column_by_name($column_name) + .unwrap() + .as_any() + .downcast_ref::<$array_type>() + .unwrap() + }; } #[test] @@ -123,11 +122,11 @@ mod tests { let result = adapt_column(&source_col, &target_field).unwrap(); let struct_array = result.as_any().downcast_ref::().unwrap(); assert_eq!(struct_array.fields().len(), 2); - let a_result = get_column_as::(&struct_array, "a"); + let a_result = get_column_as!(&struct_array, "a", Int32Array); assert_eq!(a_result.value(0), 1); assert_eq!(a_result.value(1), 2); - let b_result = get_column_as::(&struct_array, "b"); + let b_result = get_column_as!(&struct_array, "b", StringArray); assert_eq!(b_result.len(), 2); assert!(b_result.is_null(0)); assert!(b_result.is_null(1)); @@ -145,7 +144,7 @@ mod tests { let result = adapt_column(&source, &target_field).unwrap(); let struct_array = result.as_any().downcast_ref::().unwrap(); assert_eq!(struct_array.len(), 2); - let a_result = get_column_as::(&struct_array, "a"); + let a_result = get_column_as!(&struct_array, "a", Int32Array); assert_eq!(a_result.null_count(), 2); } } From 755abb5c7ae0ba2829a3f516606e0bb442872f61 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 12 Jun 2025 18:52:42 +0800 Subject: [PATCH 46/57] refactor(tests): remove unused helper functions for nested schema creation --- .../src/nested_schema_adapter/tests.rs | 59 ------------------- 1 file changed, 59 deletions(-) diff --git a/datafusion/datasource/src/nested_schema_adapter/tests.rs b/datafusion/datasource/src/nested_schema_adapter/tests.rs index d9e8aacd8da50..e95e838048033 100644 --- a/datafusion/datasource/src/nested_schema_adapter/tests.rs +++ b/datafusion/datasource/src/nested_schema_adapter/tests.rs @@ -33,65 +33,6 @@ use arrow::{ }; use datafusion_common::{ColumnStatistics, Result, ScalarValue}; use std::sync::Arc; -// ================================ -// Schema Creation Helper Functions -// ================================ - -/// Helper function to create a basic nested schema with additionalInfo -fn create_basic_nested_schema() -> SchemaRef { - Arc::new(Schema::new(vec![ - create_additional_info_field(false), // without reason field - ])) -} - -/// Helper function to create a deeply nested schema with additionalInfo including reason field -fn create_deep_nested_schema() -> SchemaRef { - Arc::new(Schema::new(vec![ - create_additional_info_field(true), // with reason field - ])) -} - -/// Helper function to create the additionalInfo field with or without the reason subfield -fn create_additional_info_field(with_reason: bool) -> Field { - let mut field_children = vec![ - Field::new("location", Utf8, true), - Field::new("timestamp_utc", Timestamp(Millisecond, None), true), - ]; - - // Add the reason field if requested (for target schema) - if with_reason { - field_children.push(create_reason_field()); - } - - Field::new("additionalInfo", Struct(field_children.into()), true) -} - -/// Helper function to create the reason nested field with its details subfield -fn create_reason_field() -> Field { - Field::new( - "reason", - Struct( - vec![ - Field::new("_level", Float64, true), - // Inline the details field creation - Field::new( - "details", - Struct( - vec![ - Field::new("rurl", Utf8, true), - Field::new("s", Float64, true), - Field::new("t", Utf8, true), - ] - .into(), - ), - true, - ), - ] - .into(), - ), - true, - ) -} // ================================ // Schema Evolution Tests From 013bf8985db854ad4335276a2d91d58444d994f6 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Fri, 13 Jun 2025 00:01:29 +0800 Subject: [PATCH 47/57] fix clippy error --- datafusion/common/src/nested_struct.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/common/src/nested_struct.rs b/datafusion/common/src/nested_struct.rs index f962b399d8230..6ff839b87044c 100644 --- a/datafusion/common/src/nested_struct.rs +++ b/datafusion/common/src/nested_struct.rs @@ -103,7 +103,7 @@ mod tests { let a_array = Arc::new(Int32Array::from(vec![1, 2])) as ArrayRef; let source_struct = StructArray::from(vec![( Arc::new(Field::new("a", DataType::Int32, true)), - a_array.clone(), + Arc::clone(&a_array), )]); let source_col = Arc::new(source_struct) as ArrayRef; From 07aa2aa7e5485ec7dc08a39f6228b7d589f5a141 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Fri, 20 Jun 2025 09:24:14 +0800 Subject: [PATCH 48/57] refactor: rename adapt_column to cast_column --- datafusion/common/src/nested_struct.rs | 20 +++++++++---------- .../src/nested_schema_adapter/adapter.rs | 4 ++-- 2 files changed, 12 insertions(+), 12 deletions(-) diff --git a/datafusion/common/src/nested_struct.rs b/datafusion/common/src/nested_struct.rs index 6ff839b87044c..6ee4764848f36 100644 --- a/datafusion/common/src/nested_struct.rs +++ b/datafusion/common/src/nested_struct.rs @@ -23,7 +23,7 @@ use arrow::{ }; use std::sync::Arc; /// Adapt a struct column to match the target field type, handling nested structs recursively -fn adapt_struct_column( +fn cast_struct_column( source_col: &ArrayRef, target_fields: &[Arc], ) -> Result { @@ -36,7 +36,7 @@ fn adapt_struct_column( match struct_array.column_by_name(target_child_field.name()) { Some(source_child_col) => { let adapted_child = - adapt_column(source_child_col, target_child_field)?; + cast_column(source_child_col, target_child_field)?; children.push((field_arc, adapted_child)); } None => { @@ -60,9 +60,9 @@ fn adapt_struct_column( } /// Adapt a column to match the target field type, handling nested structs specially -pub fn adapt_column(source_col: &ArrayRef, target_field: &Field) -> Result { +pub fn cast_column(source_col: &ArrayRef, target_field: &Field) -> Result { match target_field.data_type() { - Struct(target_fields) => adapt_struct_column(source_col, target_fields), + Struct(target_fields) => cast_struct_column(source_col, target_fields), _ => Ok(cast(source_col, target_field.data_type())?), } } @@ -87,10 +87,10 @@ mod tests { } #[test] - fn test_adapt_simple_column() { + fn test_cast_simple_column() { let source = Arc::new(Int32Array::from(vec![1, 2, 3])) as ArrayRef; let target_field = Field::new("ints", DataType::Int64, true); - let result = adapt_column(&source, &target_field).unwrap(); + let result = cast_column(&source, &target_field).unwrap(); let result = result.as_any().downcast_ref::().unwrap(); assert_eq!(result.len(), 3); assert_eq!(result.value(0), 1); @@ -99,7 +99,7 @@ mod tests { } #[test] - fn test_adapt_struct_with_missing_field() { + fn test_cast_struct_with_missing_field() { let a_array = Arc::new(Int32Array::from(vec![1, 2])) as ArrayRef; let source_struct = StructArray::from(vec![( Arc::new(Field::new("a", DataType::Int32, true)), @@ -119,7 +119,7 @@ mod tests { true, ); - let result = adapt_column(&source_col, &target_field).unwrap(); + let result = cast_column(&source_col, &target_field).unwrap(); let struct_array = result.as_any().downcast_ref::().unwrap(); assert_eq!(struct_array.fields().len(), 2); let a_result = get_column_as!(&struct_array, "a", Int32Array); @@ -133,7 +133,7 @@ mod tests { } #[test] - fn test_adapt_struct_source_not_struct() { + fn test_cast_struct_source_not_struct() { let source = Arc::new(Int32Array::from(vec![10, 20])) as ArrayRef; let target_field = Field::new( "s", @@ -141,7 +141,7 @@ mod tests { true, ); - let result = adapt_column(&source, &target_field).unwrap(); + let result = cast_column(&source, &target_field).unwrap(); let struct_array = result.as_any().downcast_ref::().unwrap(); assert_eq!(struct_array.len(), 2); let a_result = get_column_as!(&struct_array, "a", Int32Array); diff --git a/datafusion/datasource/src/nested_schema_adapter/adapter.rs b/datafusion/datasource/src/nested_schema_adapter/adapter.rs index 95af70b0c1f2d..60f46be87afdf 100644 --- a/datafusion/datasource/src/nested_schema_adapter/adapter.rs +++ b/datafusion/datasource/src/nested_schema_adapter/adapter.rs @@ -22,7 +22,7 @@ use arrow::{ array::ArrayRef, datatypes::{DataType::Struct, Field, Schema, SchemaRef}, }; -use datafusion_common::nested_struct::adapt_column; +use datafusion_common::nested_struct::cast_column; use datafusion_common::Result; use std::sync::Arc; @@ -85,7 +85,7 @@ impl SchemaAdapter for NestedStructSchemaAdapter { Arc::new(SchemaMapping::new( Arc::clone(&self.projected_table_schema), field_mappings, - Arc::new(|array: &ArrayRef, field: &Field| adapt_column(array, field)), + Arc::new(|array: &ArrayRef, field: &Field| cast_column(array, field)), )), projection, )) From e392878183bf1d1e4e6f64f7d47faff4ee7575fb Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Fri, 20 Jun 2025 09:37:07 +0800 Subject: [PATCH 49/57] refactor: cast_struct_column - error if source is not struct --- datafusion/common/src/nested_struct.rs | 83 +++++++++++++++++++++----- 1 file changed, 69 insertions(+), 14 deletions(-) diff --git a/datafusion/common/src/nested_struct.rs b/datafusion/common/src/nested_struct.rs index 6ee4764848f36..07e0c2cf34a0b 100644 --- a/datafusion/common/src/nested_struct.rs +++ b/datafusion/common/src/nested_struct.rs @@ -15,14 +15,40 @@ // specific language governing permissions and limitations // under the License. -use crate::error::Result; +use crate::error::{DataFusionError, Result}; use arrow::{ array::{new_null_array, Array, ArrayRef, StructArray}, compute::cast, - datatypes::{DataType::Struct, Field}, + datatypes::{DataType, DataType::Struct, Field}, }; use std::sync::Arc; -/// Adapt a struct column to match the target field type, handling nested structs recursively + +/// Cast a struct column to match target struct fields, handling nested structs recursively. +/// +/// This function implements struct-to-struct casting with the assumption that **structs should +/// always be allowed to cast to other structs**. However, the source column must already be +/// a struct type - non-struct sources will result in an error. +/// +/// ## Field Matching Strategy +/// - **By Name**: Source struct fields are matched to target fields by name (case-sensitive) +/// - **Type Adaptation**: When a matching field is found, it is recursively cast to the target field's type +/// - **Missing Fields**: Target fields not present in the source are filled with null values +/// - **Extra Fields**: Source fields not present in the target are ignored +/// +/// ## Nested Struct Handling +/// - Nested structs are handled recursively using the same casting rules +/// - Each level of nesting follows the same field matching and null-filling strategy +/// - This allows for complex struct transformations while maintaining data integrity +/// +/// # Arguments +/// * `source_col` - The source array to cast (must be a struct array) +/// * `target_fields` - The target struct field definitions to cast to +/// +/// # Returns +/// A `Result` containing the cast struct array +/// +/// # Errors +/// Returns a `DataFusionError::Plan` if the source column is not a struct type fn cast_struct_column( source_col: &ArrayRef, target_fields: &[Arc], @@ -51,15 +77,43 @@ fn cast_struct_column( let struct_array = StructArray::from(children); Ok(Arc::new(struct_array)) } else { - // If source is not a struct, return null array with target struct type - Ok(new_null_array( - &Struct(target_fields.to_vec().into()), - source_col.len(), - )) + // Return error if source is not a struct type + Err(DataFusionError::Plan(format!( + "Cannot cast column of type {:?} to struct type. Source must be a struct to cast to struct.", + source_col.data_type() + ))) } } -/// Adapt a column to match the target field type, handling nested structs specially +/// Cast a column to match the target field type, with special handling for nested structs. +/// +/// This function serves as the main entry point for column casting operations. For struct +/// types, it enforces that **only struct columns can be cast to struct types**. +/// +/// ## Casting Behavior +/// - **Struct Types**: Delegates to `cast_struct_column` for struct-to-struct casting only +/// - **Non-Struct Types**: Uses Arrow's standard `cast` function for primitive type conversions +/// +/// ## Struct Casting Requirements +/// The struct casting logic requires that the source column must already be a struct type. +/// This makes the function useful for: +/// - Schema evolution scenarios where struct layouts change over time +/// - Data migration between different struct schemas +/// - Type-safe data processing pipelines that maintain struct type integrity +/// +/// # Arguments +/// * `source_col` - The source array to cast +/// * `target_field` - The target field definition (including type and metadata) +/// +/// # Returns +/// A `Result` containing the cast array +/// +/// # Errors +/// Returns an error if: +/// - Attempting to cast a non-struct column to a struct type +/// - Arrow's cast function fails for non-struct types +/// - Memory allocation fails during struct construction +/// - Invalid data type combinations are encountered pub fn cast_column(source_col: &ArrayRef, target_field: &Field) -> Result { match target_field.data_type() { Struct(target_fields) => cast_struct_column(source_col, target_fields), @@ -141,10 +195,11 @@ mod tests { true, ); - let result = cast_column(&source, &target_field).unwrap(); - let struct_array = result.as_any().downcast_ref::().unwrap(); - assert_eq!(struct_array.len(), 2); - let a_result = get_column_as!(&struct_array, "a", Int32Array); - assert_eq!(a_result.null_count(), 2); + let result = cast_column(&source, &target_field); + assert!(result.is_err()); + let error_msg = result.unwrap_err().to_string(); + assert!(error_msg.contains("Cannot cast column of type")); + assert!(error_msg.contains("to struct type")); + assert!(error_msg.contains("Source must be a struct")); } } From 267768e88234369b24a3e197a1307537beaf05f1 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Fri, 20 Jun 2025 09:52:35 +0800 Subject: [PATCH 50/57] refactor: enhance struct compatibility validation for casting operations --- .../src/nested_schema_adapter/adapter.rs | 141 +++++++++++++++++- 1 file changed, 136 insertions(+), 5 deletions(-) diff --git a/datafusion/datasource/src/nested_schema_adapter/adapter.rs b/datafusion/datasource/src/nested_schema_adapter/adapter.rs index 60f46be87afdf..7eae8d74f3f0b 100644 --- a/datafusion/datasource/src/nested_schema_adapter/adapter.rs +++ b/datafusion/datasource/src/nested_schema_adapter/adapter.rs @@ -20,10 +20,10 @@ use crate::schema_adapter::{ }; use arrow::{ array::ArrayRef, - datatypes::{DataType::Struct, Field, Schema, SchemaRef}, + datatypes::{DataType::Struct, Field, FieldRef, Schema, SchemaRef}, }; use datafusion_common::nested_struct::cast_column; -use datafusion_common::Result; +use datafusion_common::{plan_err, Result}; use std::sync::Arc; /// A SchemaAdapter that handles schema evolution for nested struct types @@ -72,10 +72,11 @@ impl SchemaAdapter for NestedStructSchemaAdapter { file_schema, &self.projected_table_schema, |file_field, table_field| { - // Special handling for struct fields - always include them even if the - // internal structure differs, as we'll adapt them later + // Special handling for struct fields - validate internal structure compatibility match (file_field.data_type(), table_field.data_type()) { - (Struct(_), Struct(_)) => Ok(true), + (Struct(source_fields), Struct(target_fields)) => { + validate_struct_compatibility(source_fields, target_fields) + } _ => crate::schema_adapter::can_cast_field(file_field, table_field), } }, @@ -91,3 +92,133 @@ impl SchemaAdapter for NestedStructSchemaAdapter { )) } } + +/// Validates compatibility between source and target struct fields for casting operations. +/// +/// This function implements comprehensive struct compatibility checking by examining: +/// - Field name matching between source and target structs +/// - Type castability for each matching field (including recursive struct validation) +/// - Proper handling of missing fields (target fields not in source are allowed - filled with nulls) +/// - Proper handling of extra fields (source fields not in target are allowed - ignored) +/// +/// # Compatibility Rules +/// - **Field Matching**: Fields are matched by name (case-sensitive) +/// - **Missing Target Fields**: Allowed - will be filled with null values during casting +/// - **Extra Source Fields**: Allowed - will be ignored during casting +/// - **Type Compatibility**: Each matching field must be castable using Arrow's type system +/// - **Nested Structs**: Recursively validates nested struct compatibility +/// +/// # Arguments +/// * `source_fields` - Fields from the source struct type +/// * `target_fields` - Fields from the target struct type +/// +/// # Returns +/// * `Ok(true)` if the structs are compatible for casting +/// * `Err(DataFusionError)` with detailed error message if incompatible +/// +/// # Examples +/// ```ignore +/// // Compatible: source has extra field, target has missing field +/// // Source: {a: i32, b: string, c: f64} +/// // Target: {a: i64, d: bool} +/// // Result: Ok(true) - 'a' can cast i32->i64, 'b','c' ignored, 'd' filled with nulls +/// +/// // Incompatible: matching field has incompatible types +/// // Source: {a: string} +/// // Target: {a: binary} +/// // Result: Err(...) - string cannot cast to binary +/// ``` +fn validate_struct_compatibility( + source_fields: &[FieldRef], + target_fields: &[FieldRef], +) -> Result { + // Check compatibility for each target field + for target_field in target_fields { + // Look for matching field in source by name + if let Some(source_field) = source_fields + .iter() + .find(|f| f.name() == target_field.name()) + { + // Check if the matching field types are compatible + match (source_field.data_type(), target_field.data_type()) { + // Recursively validate nested structs + (Struct(source_nested), Struct(target_nested)) => { + validate_struct_compatibility(source_nested, target_nested)?; + } + // For non-struct types, use the existing castability check + _ => { + if !arrow::compute::can_cast_types( + source_field.data_type(), + target_field.data_type(), + ) { + return plan_err!( + "Cannot cast struct field '{}' from type {:?} to type {:?}", + target_field.name(), + source_field.data_type(), + target_field.data_type() + ); + } + } + } + } + // Missing fields in source are OK - they'll be filled with nulls + } + + // Extra fields in source are OK - they'll be ignored + Ok(true) +} + +#[cfg(test)] +mod tests { + use super::*; + use arrow::datatypes::{DataType, Field}; + + #[test] + fn test_validate_struct_compatibility_incompatible_types() { + // Source struct: {field1: String, field2: String} + let source_fields = vec![ + Arc::new(Field::new("field1", DataType::Utf8, true)), + Arc::new(Field::new("field2", DataType::Utf8, true)), + ]; + + // Target struct: {field1: UInt32} + let target_fields = vec![Arc::new(Field::new("field1", DataType::UInt32, true))]; + + let result = validate_struct_compatibility(&source_fields, &target_fields); + assert!(result.is_err()); + let error_msg = result.unwrap_err().to_string(); + assert!(error_msg.contains("Cannot cast struct field 'field1'")); + assert!(error_msg.contains("Utf8")); + assert!(error_msg.contains("UInt32")); + } + + #[test] + fn test_validate_struct_compatibility_compatible_types() { + // Source struct: {field1: Int32, field2: String} + let source_fields = vec![ + Arc::new(Field::new("field1", DataType::Int32, true)), + Arc::new(Field::new("field2", DataType::Utf8, true)), + ]; + + // Target struct: {field1: Int64} (Int32 can cast to Int64) + let target_fields = vec![Arc::new(Field::new("field1", DataType::Int64, true))]; + + let result = validate_struct_compatibility(&source_fields, &target_fields); + assert!(result.is_ok()); + assert_eq!(result.unwrap(), true); + } + + #[test] + fn test_validate_struct_compatibility_missing_field_in_source() { + // Source struct: {field2: String} (missing field1) + let source_fields = vec![Arc::new(Field::new("field2", DataType::Utf8, true))]; + + // Target struct: {field1: Int32} + let target_fields = vec![Arc::new(Field::new("field1", DataType::Int32, true))]; + + // Should be OK - missing fields will be filled with nulls + let result = validate_struct_compatibility(&source_fields, &target_fields); + assert!(result.is_ok()); + assert_eq!(result.unwrap(), true); + } +} From 53ab183330b68722fb53cc2b0d2e104c28e8818c Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Fri, 20 Jun 2025 12:44:33 +0800 Subject: [PATCH 51/57] remove unused import --- datafusion/common/src/nested_struct.rs | 26 +++++++++++++------------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/datafusion/common/src/nested_struct.rs b/datafusion/common/src/nested_struct.rs index 07e0c2cf34a0b..a68ba26b071de 100644 --- a/datafusion/common/src/nested_struct.rs +++ b/datafusion/common/src/nested_struct.rs @@ -19,34 +19,34 @@ use crate::error::{DataFusionError, Result}; use arrow::{ array::{new_null_array, Array, ArrayRef, StructArray}, compute::cast, - datatypes::{DataType, DataType::Struct, Field}, + datatypes::{DataType::Struct, Field}, }; use std::sync::Arc; /// Cast a struct column to match target struct fields, handling nested structs recursively. -/// +/// /// This function implements struct-to-struct casting with the assumption that **structs should /// always be allowed to cast to other structs**. However, the source column must already be /// a struct type - non-struct sources will result in an error. -/// +/// /// ## Field Matching Strategy /// - **By Name**: Source struct fields are matched to target fields by name (case-sensitive) /// - **Type Adaptation**: When a matching field is found, it is recursively cast to the target field's type /// - **Missing Fields**: Target fields not present in the source are filled with null values /// - **Extra Fields**: Source fields not present in the target are ignored -/// +/// /// ## Nested Struct Handling /// - Nested structs are handled recursively using the same casting rules /// - Each level of nesting follows the same field matching and null-filling strategy /// - This allows for complex struct transformations while maintaining data integrity -/// +/// /// # Arguments /// * `source_col` - The source array to cast (must be a struct array) /// * `target_fields` - The target struct field definitions to cast to -/// +/// /// # Returns /// A `Result` containing the cast struct array -/// +/// /// # Errors /// Returns a `DataFusionError::Plan` if the source column is not a struct type fn cast_struct_column( @@ -86,28 +86,28 @@ fn cast_struct_column( } /// Cast a column to match the target field type, with special handling for nested structs. -/// +/// /// This function serves as the main entry point for column casting operations. For struct /// types, it enforces that **only struct columns can be cast to struct types**. -/// +/// /// ## Casting Behavior /// - **Struct Types**: Delegates to `cast_struct_column` for struct-to-struct casting only /// - **Non-Struct Types**: Uses Arrow's standard `cast` function for primitive type conversions -/// +/// /// ## Struct Casting Requirements /// The struct casting logic requires that the source column must already be a struct type. /// This makes the function useful for: /// - Schema evolution scenarios where struct layouts change over time /// - Data migration between different struct schemas /// - Type-safe data processing pipelines that maintain struct type integrity -/// +/// /// # Arguments /// * `source_col` - The source array to cast /// * `target_field` - The target field definition (including type and metadata) -/// +/// /// # Returns /// A `Result` containing the cast array -/// +/// /// # Errors /// Returns an error if: /// - Attempting to cast a non-struct column to a struct type From 3a766fd52366db474348d3eb073375adae56e270 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Fri, 20 Jun 2025 12:48:03 +0800 Subject: [PATCH 52/57] refactor: remove schema adapter factory and related code from ListingTableConfig --- .../core/src/datasource/listing/table.rs | 336 ++---------------- 1 file changed, 20 insertions(+), 316 deletions(-) diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index ecf5c18cc1602..3ddf1c85e241b 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -36,10 +36,9 @@ use datafusion_common::{ }; use datafusion_datasource::{ compute_all_files_statistics, - file::FileSource, file_groups::FileGroup, file_scan_config::{FileScanConfig, FileScanConfigBuilder}, - schema_adapter::{DefaultSchemaAdapterFactory, SchemaAdapter, SchemaAdapterFactory}, + schema_adapter::DefaultSchemaAdapterFactory, }; use datafusion_execution::{ cache::{cache_manager::FileStatisticsCache, cache_unit::DefaultFileStatisticsCache}, @@ -56,10 +55,9 @@ use object_store::ObjectStore; use std::{any::Any, collections::HashMap, str::FromStr, sync::Arc}; /// Indicates the source of the schema for a [`ListingTable`] // PartialEq required for assert_eq! in tests -#[derive(Debug, Clone, Copy, PartialEq, Default)] +#[derive(Debug, Clone, Copy, PartialEq)] pub enum SchemaSource { /// Schema is not yet set (initial state) - #[default] None, /// Schema was inferred from first table_path Inferred, @@ -70,7 +68,7 @@ pub enum SchemaSource { /// Configuration for creating a [`ListingTable`] /// /// -#[derive(Debug, Clone, Default)] +#[derive(Debug, Clone)] pub struct ListingTableConfig { /// Paths on the `ObjectStore` for creating `ListingTable`. /// They should share the same schema and object store. @@ -85,16 +83,17 @@ pub struct ListingTableConfig { pub options: Option, /// Tracks the source of the schema information schema_source: SchemaSource, - /// Optional [`SchemaAdapterFactory`] for creating schema adapters - schema_adapter_factory: Option>, } impl ListingTableConfig { /// Creates new [`ListingTableConfig`] for reading the specified URL pub fn new(table_path: ListingTableUrl) -> Self { + let table_paths = vec![table_path]; Self { - table_paths: vec![table_path], - ..Default::default() + table_paths, + file_schema: None, + options: None, + schema_source: SchemaSource::None, } } @@ -104,7 +103,9 @@ impl ListingTableConfig { pub fn new_with_multi_paths(table_paths: Vec) -> Self { Self { table_paths, - ..Default::default() + file_schema: None, + options: None, + schema_source: SchemaSource::None, } } @@ -124,9 +125,10 @@ impl ListingTableConfig { /// without the table partitioning columns. pub fn with_schema(self, schema: SchemaRef) -> Self { Self { + table_paths: self.table_paths, file_schema: Some(schema), + options: self.options, schema_source: SchemaSource::Specified, - ..self } } @@ -136,8 +138,10 @@ impl ListingTableConfig { /// [`Self::infer_options`]. pub fn with_listing_options(self, listing_options: ListingOptions) -> Self { Self { + table_paths: self.table_paths, + file_schema: self.file_schema, options: Some(listing_options), - ..self + schema_source: self.schema_source, } } @@ -218,7 +222,6 @@ impl ListingTableConfig { file_schema: self.file_schema, options: Some(listing_options), schema_source: self.schema_source, - schema_adapter_factory: self.schema_adapter_factory, }) } @@ -237,7 +240,6 @@ impl ListingTableConfig { file_schema, options: _, schema_source, - schema_adapter_factory, } = self; let (schema, new_schema_source) = match file_schema { @@ -259,7 +261,6 @@ impl ListingTableConfig { file_schema: Some(schema), options: Some(options), schema_source: new_schema_source, - schema_adapter_factory, }) } None => internal_err!("No `ListingOptions` set for inferring schema"), @@ -301,34 +302,11 @@ impl ListingTableConfig { file_schema: self.file_schema, options: Some(options), schema_source: self.schema_source, - schema_adapter_factory: self.schema_adapter_factory, }) } None => config_err!("No `ListingOptions` set for inferring schema"), } } - - /// Set the [`SchemaAdapterFactory`] for the [`ListingTable`] - /// - /// The schema adapter factory is used to create schema adapters that can - /// handle schema evolution and type conversions when reading files with - /// different schemas than the table schema. - /// - /// If not provided, a default schema adapter factory will be used. - pub fn with_schema_adapter_factory( - self, - schema_adapter_factory: Arc, - ) -> Self { - Self { - schema_adapter_factory: Some(schema_adapter_factory), - ..self - } - } - - /// Get the [`SchemaAdapterFactory`] for this configuration - pub fn schema_adapter_factory(&self) -> Option<&Arc> { - self.schema_adapter_factory.as_ref() - } } /// Options for creating a [`ListingTable`] @@ -823,8 +801,6 @@ pub struct ListingTable { collected_statistics: FileStatisticsCache, constraints: Constraints, column_defaults: HashMap, - /// Optional [`SchemaAdapterFactory`] for creating schema adapters - schema_adapter_factory: Option>, } impl ListingTable { @@ -865,7 +841,6 @@ impl ListingTable { collected_statistics: Arc::new(DefaultFileStatisticsCache::default()), constraints: Constraints::default(), column_defaults: HashMap::new(), - schema_adapter_factory: config.schema_adapter_factory, }; Ok(table) @@ -919,50 +894,6 @@ impl ListingTable { self.schema_source } - /// Set the [`SchemaAdapterFactory`] for this [`ListingTable`] - /// - /// The schema adapter factory is used to create schema adapters that can - /// handle schema evolution and type conversions when reading files with - /// different schemas than the table schema. - pub fn with_schema_adapter_factory( - self, - schema_adapter_factory: Arc, - ) -> Self { - Self { - schema_adapter_factory: Some(schema_adapter_factory), - ..self - } - } - - /// Get the [`SchemaAdapterFactory`] for this table - pub fn schema_adapter_factory(&self) -> Option<&Arc> { - self.schema_adapter_factory.as_ref() - } - - /// Creates a schema adapter for mapping between file and table schemas - /// - /// Uses the configured schema adapter factory if available, otherwise falls back - /// to the default implementation. - fn create_schema_adapter(&self) -> Box { - let table_schema = self.schema(); - match &self.schema_adapter_factory { - Some(factory) => { - factory.create(Arc::clone(&table_schema), Arc::clone(&table_schema)) - } - None => DefaultSchemaAdapterFactory::from_schema(Arc::clone(&table_schema)), - } - } - - /// Creates a file source and applies schema adapter factory if available - fn create_file_source_with_schema_adapter(&self) -> Result> { - let mut source = self.options.format.file_source(); - // Apply schema adapter to source if available - if let Some(factory) = &self.schema_adapter_factory { - source = source.with_schema_adapter_factory(Arc::clone(factory))?; - } - Ok(source) - } - /// If file_sort_order is specified, creates the appropriate physical expressions fn try_create_output_ordering(&self) -> Result> { create_ordering(&self.table_schema, &self.options.file_sort_order) @@ -1071,8 +1002,6 @@ impl TableProvider for ListingTable { return Ok(Arc::new(EmptyExec::new(Arc::new(Schema::empty())))); }; - let file_source = self.create_file_source_with_schema_adapter()?; - // create the execution plan self.options .format @@ -1081,7 +1010,7 @@ impl TableProvider for ListingTable { FileScanConfigBuilder::new( object_store_url, Arc::clone(&self.file_schema), - file_source, + self.options.format.file_source(), ) .with_file_groups(partitioned_file_lists) .with_constraints(self.constraints.clone()) @@ -1240,10 +1169,8 @@ impl ListingTable { self.options.collect_stat, inexact_stats, )?; - - let schema_adapter = self.create_schema_adapter(); - let (schema_mapper, _) = schema_adapter.map_schema(self.file_schema.as_ref())?; - + let (schema_mapper, _) = DefaultSchemaAdapterFactory::from_schema(self.schema()) + .map_schema(self.file_schema.as_ref())?; stats.column_statistics = schema_mapper.map_column_statistics(&stats.column_statistics)?; file_groups.iter_mut().try_for_each(|file_group| { @@ -1393,21 +1320,15 @@ mod tests { assert_contains, stats::Precision, test_util::{batches_to_string, datafusion_test_data}, - ColumnStatistics, ScalarValue, - }; - use datafusion_datasource::schema_adapter::{ - SchemaAdapter, SchemaAdapterFactory, SchemaMapper, + ScalarValue, }; use datafusion_expr::{BinaryExpr, LogicalPlanBuilder, Operator}; use datafusion_physical_expr::PhysicalSortExpr; use datafusion_physical_plan::{collect, ExecutionPlanProperties}; - use rstest::rstest; use std::io::Write; use tempfile::TempDir; use url::Url; - const DUMMY_NULL_COUNT: Precision = Precision::Exact(42); - /// Creates a test schema with standard field types used in tests fn create_test_schema() -> SchemaRef { Arc::new(Schema::new(vec![ @@ -2632,221 +2553,4 @@ mod tests { Ok(()) } - - #[tokio::test] - async fn test_statistics_mapping_with_custom_factory() -> Result<()> { - let ctx = SessionContext::new(); - let table = create_test_listing_table_with_json_and_adapter( - &ctx, - false, - // NullStatsAdapterFactory sets column_statistics null_count to DUMMY_NULL_COUNT - Arc::new(NullStatsAdapterFactory {}), - )?; - - let (groups, stats) = table.list_files_for_scan(&ctx.state(), &[], None).await?; - - assert_eq!(stats.column_statistics[0].null_count, DUMMY_NULL_COUNT); - for g in groups { - if let Some(s) = g.file_statistics(None) { - assert_eq!(s.column_statistics[0].null_count, DUMMY_NULL_COUNT); - } - } - - Ok(()) - } - - #[rstest] - #[case(MapSchemaError::TypeIncompatible, "Cannot map incompatible types")] - #[case(MapSchemaError::GeneralFailure, "Schema adapter mapping failed")] - #[case( - MapSchemaError::InvalidProjection, - "Invalid projection in schema mapping" - )] - #[tokio::test] - async fn test_schema_adapter_map_schema_errors( - #[case] error_type: MapSchemaError, - #[case] expected_error_msg: &str, - ) -> Result<()> { - let ctx = SessionContext::new(); - let table = create_test_listing_table_with_json_and_adapter( - &ctx, - false, - Arc::new(FailingMapSchemaAdapterFactory { error_type }), - )?; - - // The error should bubble up from the scan operation when schema mapping fails - let scan_result = table.scan(&ctx.state(), None, &[], None).await; - - assert!(scan_result.is_err()); - let error_msg = scan_result.unwrap_err().to_string(); - assert!( - error_msg.contains(expected_error_msg), - "Expected error containing '{expected_error_msg}', got: {error_msg}" - ); - - Ok(()) - } - - // Test that errors during file listing also bubble up correctly - #[tokio::test] - async fn test_schema_adapter_error_during_file_listing() -> Result<()> { - let ctx = SessionContext::new(); - let table = create_test_listing_table_with_json_and_adapter( - &ctx, - true, - Arc::new(FailingMapSchemaAdapterFactory { - error_type: MapSchemaError::TypeIncompatible, - }), - )?; - - // The error should bubble up from list_files_for_scan when collecting statistics - let list_result = table.list_files_for_scan(&ctx.state(), &[], None).await; - - assert!(list_result.is_err()); - let error_msg = list_result.unwrap_err().to_string(); - assert!( - error_msg.contains("Cannot map incompatible types"), - "Expected type incompatibility error during file listing, got: {error_msg}" - ); - - Ok(()) - } - - #[derive(Debug, Copy, Clone)] - enum MapSchemaError { - TypeIncompatible, - GeneralFailure, - InvalidProjection, - } - - #[derive(Debug)] - struct FailingMapSchemaAdapterFactory { - error_type: MapSchemaError, - } - - impl SchemaAdapterFactory for FailingMapSchemaAdapterFactory { - fn create( - &self, - projected_table_schema: SchemaRef, - _table_schema: SchemaRef, - ) -> Box { - Box::new(FailingMapSchemaAdapter { - schema: projected_table_schema, - error_type: self.error_type, - }) - } - } - - #[derive(Debug)] - struct FailingMapSchemaAdapter { - schema: SchemaRef, - error_type: MapSchemaError, - } - - impl SchemaAdapter for FailingMapSchemaAdapter { - fn map_column_index(&self, index: usize, file_schema: &Schema) -> Option { - let field = self.schema.field(index); - file_schema.fields.find(field.name()).map(|(i, _)| i) - } - - fn map_schema( - &self, - _file_schema: &Schema, - ) -> Result<(Arc, Vec)> { - // Always fail with different error types based on the configured error_type - match self.error_type { - MapSchemaError::TypeIncompatible => { - plan_err!( - "Cannot map incompatible types: Boolean cannot be cast to Utf8" - ) - } - MapSchemaError::GeneralFailure => { - plan_err!("Schema adapter mapping failed due to internal error") - } - MapSchemaError::InvalidProjection => { - plan_err!("Invalid projection in schema mapping: column index out of bounds") - } - } - } - } - - #[derive(Debug)] - struct NullStatsAdapterFactory; - - impl SchemaAdapterFactory for NullStatsAdapterFactory { - fn create( - &self, - projected_table_schema: SchemaRef, - _table_schema: SchemaRef, - ) -> Box { - Box::new(NullStatsAdapter { - schema: projected_table_schema, - }) - } - } - - #[derive(Debug)] - struct NullStatsAdapter { - schema: SchemaRef, - } - - impl SchemaAdapter for NullStatsAdapter { - fn map_column_index(&self, index: usize, file_schema: &Schema) -> Option { - let field = self.schema.field(index); - file_schema.fields.find(field.name()).map(|(i, _)| i) - } - - fn map_schema( - &self, - file_schema: &Schema, - ) -> Result<(Arc, Vec)> { - let projection = (0..file_schema.fields().len()).collect(); - Ok((Arc::new(NullStatsMapper {}), projection)) - } - } - - #[derive(Debug)] - struct NullStatsMapper; - - impl SchemaMapper for NullStatsMapper { - fn map_batch(&self, batch: RecordBatch) -> Result { - Ok(batch) - } - - fn map_column_statistics( - &self, - stats: &[ColumnStatistics], - ) -> Result> { - Ok(stats - .iter() - .map(|s| { - let mut s = s.clone(); - s.null_count = DUMMY_NULL_COUNT; - s - }) - .collect()) - } - } - - /// Helper function to create a test ListingTable with JSON format and custom schema adapter factory - fn create_test_listing_table_with_json_and_adapter( - ctx: &SessionContext, - collect_stat: bool, - schema_adapter_factory: Arc, - ) -> Result { - let path = "table/file.json"; - register_test_store(ctx, &[(path, 10)]); - - let format = JsonFormat::default(); - let opt = ListingOptions::new(Arc::new(format)).with_collect_stat(collect_stat); - let schema = Schema::new(vec![Field::new("a", DataType::Boolean, false)]); - let table_path = ListingTableUrl::parse("test:///table/").unwrap(); - - let config = ListingTableConfig::new(table_path) - .with_listing_options(opt) - .with_schema(Arc::new(schema)) - .with_schema_adapter_factory(schema_adapter_factory); - - ListingTable::try_new(config) - } } From 6fa8c8916a97d19e32a9b4a0da8c1176b694588f Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Fri, 20 Jun 2025 13:12:57 +0800 Subject: [PATCH 53/57] fix: tests --- .../src/nested_schema_adapter/adapter.rs | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/datafusion/datasource/src/nested_schema_adapter/adapter.rs b/datafusion/datasource/src/nested_schema_adapter/adapter.rs index 7eae8d74f3f0b..83e5605d14220 100644 --- a/datafusion/datasource/src/nested_schema_adapter/adapter.rs +++ b/datafusion/datasource/src/nested_schema_adapter/adapter.rs @@ -175,21 +175,21 @@ mod tests { #[test] fn test_validate_struct_compatibility_incompatible_types() { - // Source struct: {field1: String, field2: String} + // Source struct: {field1: Binary, field2: String} let source_fields = vec![ - Arc::new(Field::new("field1", DataType::Utf8, true)), + Arc::new(Field::new("field1", DataType::Binary, true)), Arc::new(Field::new("field2", DataType::Utf8, true)), ]; - // Target struct: {field1: UInt32} - let target_fields = vec![Arc::new(Field::new("field1", DataType::UInt32, true))]; + // Target struct: {field1: Int32} + let target_fields = vec![Arc::new(Field::new("field1", DataType::Int32, true))]; let result = validate_struct_compatibility(&source_fields, &target_fields); assert!(result.is_err()); let error_msg = result.unwrap_err().to_string(); assert!(error_msg.contains("Cannot cast struct field 'field1'")); - assert!(error_msg.contains("Utf8")); - assert!(error_msg.contains("UInt32")); + assert!(error_msg.contains("Binary")); + assert!(error_msg.contains("Int32")); } #[test] @@ -205,7 +205,7 @@ mod tests { let result = validate_struct_compatibility(&source_fields, &target_fields); assert!(result.is_ok()); - assert_eq!(result.unwrap(), true); + assert!(result.unwrap()); } #[test] @@ -219,6 +219,6 @@ mod tests { // Should be OK - missing fields will be filled with nulls let result = validate_struct_compatibility(&source_fields, &target_fields); assert!(result.is_ok()); - assert_eq!(result.unwrap(), true); + assert!(result.unwrap()); } } From 96f97b75a431c4dfb781050407b9b921401e4ac2 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Fri, 20 Jun 2025 13:24:48 +0800 Subject: [PATCH 54/57] refactor: rename AdaptColumnFn to CastColumnFn for clarity --- datafusion/datasource/src/schema_adapter.rs | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/datafusion/datasource/src/schema_adapter.rs b/datafusion/datasource/src/schema_adapter.rs index 5195f6e6079e6..c869f1ac19230 100644 --- a/datafusion/datasource/src/schema_adapter.rs +++ b/datafusion/datasource/src/schema_adapter.rs @@ -29,7 +29,7 @@ use datafusion_common::{plan_err, ColumnStatistics}; use std::{fmt::Debug, sync::Arc}; /// Function used by [`SchemaMapping`] to adapt a column from the file schema to /// the table schema. -pub type AdaptColumnFn = +pub type CastColumnFn = dyn Fn(&ArrayRef, &Field) -> datafusion_common::Result + Send + Sync; /// Factory for creating [`SchemaAdapter`] @@ -342,7 +342,7 @@ pub struct SchemaMapping { field_mappings: Vec>, /// Function used to adapt a column from the file schema to the table schema /// when it exists in both schemas - adapt_column: Arc, + cast_column: Arc, } impl Debug for SchemaMapping { @@ -350,7 +350,7 @@ impl Debug for SchemaMapping { f.debug_struct("SchemaMapping") .field("projected_table_schema", &self.projected_table_schema) .field("field_mappings", &self.field_mappings) - .field("adapt_column", &"") + .field("cast_column", &"") .finish() } } @@ -362,12 +362,12 @@ impl SchemaMapping { pub fn new( projected_table_schema: SchemaRef, field_mappings: Vec>, - adapt_column: Arc, + cast_column: Arc, ) -> Self { Self { projected_table_schema, field_mappings, - adapt_column, + cast_column, } } } @@ -394,9 +394,9 @@ impl SchemaMapper for SchemaMapping { // If this field only exists in the table, and not in the file, then we know // that it's null, so just return that. || Ok(new_null_array(field.data_type(), batch_rows)), - // However, if it does exist in both, use the adapt_column function + // However, if it does exist in both, use the cast_column function // to perform any necessary conversions - |batch_idx| (self.adapt_column)(&batch_cols[batch_idx], field), + |batch_idx| (self.cast_column)(&batch_cols[batch_idx], field), ) }) .collect::, _>>()?; From 2bc8256d6dbf3e953642c20fe53f416aa442d9a6 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Fri, 20 Jun 2025 14:34:57 +0800 Subject: [PATCH 55/57] feat: move and make validate_struct_compatibility public for nested struct casting --- datafusion/common/src/nested_struct.rs | 128 ++++++++++++++++- .../src/nested_schema_adapter/adapter.rs | 136 +----------------- 2 files changed, 129 insertions(+), 135 deletions(-) diff --git a/datafusion/common/src/nested_struct.rs b/datafusion/common/src/nested_struct.rs index a68ba26b071de..d5c215da41ff4 100644 --- a/datafusion/common/src/nested_struct.rs +++ b/datafusion/common/src/nested_struct.rs @@ -15,11 +15,11 @@ // specific language governing permissions and limitations // under the License. -use crate::error::{DataFusionError, Result}; +use crate::error::{DataFusionError, Result, _plan_err}; use arrow::{ array::{new_null_array, Array, ArrayRef, StructArray}, compute::cast, - datatypes::{DataType::Struct, Field}, + datatypes::{DataType::Struct, Field, FieldRef}, }; use std::sync::Arc; @@ -121,6 +121,81 @@ pub fn cast_column(source_col: &ArrayRef, target_field: &Field) -> Resulti64, 'b','c' ignored, 'd' filled with nulls +/// +/// // Incompatible: matching field has incompatible types +/// // Source: {a: string} +/// // Target: {a: binary} +/// // Result: Err(...) - string cannot cast to binary +/// ``` +pub fn validate_struct_compatibility( + source_fields: &[FieldRef], + target_fields: &[FieldRef], +) -> Result { + // Check compatibility for each target field + for target_field in target_fields { + // Look for matching field in source by name + if let Some(source_field) = source_fields + .iter() + .find(|f| f.name() == target_field.name()) + { + // Check if the matching field types are compatible + match (source_field.data_type(), target_field.data_type()) { + // Recursively validate nested structs + (Struct(source_nested), Struct(target_nested)) => { + validate_struct_compatibility(source_nested, target_nested)?; + } + // For non-struct types, use the existing castability check + _ => { + if !arrow::compute::can_cast_types( + source_field.data_type(), + target_field.data_type(), + ) { + return _plan_err!( + "Cannot cast struct field '{}' from type {:?} to type {:?}", + target_field.name(), + source_field.data_type(), + target_field.data_type() + ); + } + } + } + } + // Missing fields in source are OK - they'll be filled with nulls + } + + // Extra fields in source are OK - they'll be ignored + Ok(true) +} + #[cfg(test)] mod tests { use super::*; @@ -202,4 +277,53 @@ mod tests { assert!(error_msg.contains("to struct type")); assert!(error_msg.contains("Source must be a struct")); } + + #[test] + fn test_validate_struct_compatibility_incompatible_types() { + // Source struct: {field1: Binary, field2: String} + let source_fields = vec![ + Arc::new(Field::new("field1", DataType::Binary, true)), + Arc::new(Field::new("field2", DataType::Utf8, true)), + ]; + + // Target struct: {field1: Int32} + let target_fields = vec![Arc::new(Field::new("field1", DataType::Int32, true))]; + + let result = validate_struct_compatibility(&source_fields, &target_fields); + assert!(result.is_err()); + let error_msg = result.unwrap_err().to_string(); + assert!(error_msg.contains("Cannot cast struct field 'field1'")); + assert!(error_msg.contains("Binary")); + assert!(error_msg.contains("Int32")); + } + + #[test] + fn test_validate_struct_compatibility_compatible_types() { + // Source struct: {field1: Int32, field2: String} + let source_fields = vec![ + Arc::new(Field::new("field1", DataType::Int32, true)), + Arc::new(Field::new("field2", DataType::Utf8, true)), + ]; + + // Target struct: {field1: Int64} (Int32 can cast to Int64) + let target_fields = vec![Arc::new(Field::new("field1", DataType::Int64, true))]; + + let result = validate_struct_compatibility(&source_fields, &target_fields); + assert!(result.is_ok()); + assert!(result.unwrap()); + } + + #[test] + fn test_validate_struct_compatibility_missing_field_in_source() { + // Source struct: {field2: String} (missing field1) + let source_fields = vec![Arc::new(Field::new("field2", DataType::Utf8, true))]; + + // Target struct: {field1: Int32} + let target_fields = vec![Arc::new(Field::new("field1", DataType::Int32, true))]; + + // Should be OK - missing fields will be filled with nulls + let result = validate_struct_compatibility(&source_fields, &target_fields); + assert!(result.is_ok()); + assert!(result.unwrap()); + } } diff --git a/datafusion/datasource/src/nested_schema_adapter/adapter.rs b/datafusion/datasource/src/nested_schema_adapter/adapter.rs index 83e5605d14220..0e1e2c441bb13 100644 --- a/datafusion/datasource/src/nested_schema_adapter/adapter.rs +++ b/datafusion/datasource/src/nested_schema_adapter/adapter.rs @@ -20,10 +20,10 @@ use crate::schema_adapter::{ }; use arrow::{ array::ArrayRef, - datatypes::{DataType::Struct, Field, FieldRef, Schema, SchemaRef}, + datatypes::{DataType::Struct, Field, Schema, SchemaRef}, }; -use datafusion_common::nested_struct::cast_column; -use datafusion_common::{plan_err, Result}; +use datafusion_common::nested_struct::{cast_column, validate_struct_compatibility}; +use datafusion_common::Result; use std::sync::Arc; /// A SchemaAdapter that handles schema evolution for nested struct types @@ -92,133 +92,3 @@ impl SchemaAdapter for NestedStructSchemaAdapter { )) } } - -/// Validates compatibility between source and target struct fields for casting operations. -/// -/// This function implements comprehensive struct compatibility checking by examining: -/// - Field name matching between source and target structs -/// - Type castability for each matching field (including recursive struct validation) -/// - Proper handling of missing fields (target fields not in source are allowed - filled with nulls) -/// - Proper handling of extra fields (source fields not in target are allowed - ignored) -/// -/// # Compatibility Rules -/// - **Field Matching**: Fields are matched by name (case-sensitive) -/// - **Missing Target Fields**: Allowed - will be filled with null values during casting -/// - **Extra Source Fields**: Allowed - will be ignored during casting -/// - **Type Compatibility**: Each matching field must be castable using Arrow's type system -/// - **Nested Structs**: Recursively validates nested struct compatibility -/// -/// # Arguments -/// * `source_fields` - Fields from the source struct type -/// * `target_fields` - Fields from the target struct type -/// -/// # Returns -/// * `Ok(true)` if the structs are compatible for casting -/// * `Err(DataFusionError)` with detailed error message if incompatible -/// -/// # Examples -/// ```ignore -/// // Compatible: source has extra field, target has missing field -/// // Source: {a: i32, b: string, c: f64} -/// // Target: {a: i64, d: bool} -/// // Result: Ok(true) - 'a' can cast i32->i64, 'b','c' ignored, 'd' filled with nulls -/// -/// // Incompatible: matching field has incompatible types -/// // Source: {a: string} -/// // Target: {a: binary} -/// // Result: Err(...) - string cannot cast to binary -/// ``` -fn validate_struct_compatibility( - source_fields: &[FieldRef], - target_fields: &[FieldRef], -) -> Result { - // Check compatibility for each target field - for target_field in target_fields { - // Look for matching field in source by name - if let Some(source_field) = source_fields - .iter() - .find(|f| f.name() == target_field.name()) - { - // Check if the matching field types are compatible - match (source_field.data_type(), target_field.data_type()) { - // Recursively validate nested structs - (Struct(source_nested), Struct(target_nested)) => { - validate_struct_compatibility(source_nested, target_nested)?; - } - // For non-struct types, use the existing castability check - _ => { - if !arrow::compute::can_cast_types( - source_field.data_type(), - target_field.data_type(), - ) { - return plan_err!( - "Cannot cast struct field '{}' from type {:?} to type {:?}", - target_field.name(), - source_field.data_type(), - target_field.data_type() - ); - } - } - } - } - // Missing fields in source are OK - they'll be filled with nulls - } - - // Extra fields in source are OK - they'll be ignored - Ok(true) -} - -#[cfg(test)] -mod tests { - use super::*; - use arrow::datatypes::{DataType, Field}; - - #[test] - fn test_validate_struct_compatibility_incompatible_types() { - // Source struct: {field1: Binary, field2: String} - let source_fields = vec![ - Arc::new(Field::new("field1", DataType::Binary, true)), - Arc::new(Field::new("field2", DataType::Utf8, true)), - ]; - - // Target struct: {field1: Int32} - let target_fields = vec![Arc::new(Field::new("field1", DataType::Int32, true))]; - - let result = validate_struct_compatibility(&source_fields, &target_fields); - assert!(result.is_err()); - let error_msg = result.unwrap_err().to_string(); - assert!(error_msg.contains("Cannot cast struct field 'field1'")); - assert!(error_msg.contains("Binary")); - assert!(error_msg.contains("Int32")); - } - - #[test] - fn test_validate_struct_compatibility_compatible_types() { - // Source struct: {field1: Int32, field2: String} - let source_fields = vec![ - Arc::new(Field::new("field1", DataType::Int32, true)), - Arc::new(Field::new("field2", DataType::Utf8, true)), - ]; - - // Target struct: {field1: Int64} (Int32 can cast to Int64) - let target_fields = vec![Arc::new(Field::new("field1", DataType::Int64, true))]; - - let result = validate_struct_compatibility(&source_fields, &target_fields); - assert!(result.is_ok()); - assert!(result.unwrap()); - } - - #[test] - fn test_validate_struct_compatibility_missing_field_in_source() { - // Source struct: {field2: String} (missing field1) - let source_fields = vec![Arc::new(Field::new("field2", DataType::Utf8, true))]; - - // Target struct: {field1: Int32} - let target_fields = vec![Arc::new(Field::new("field1", DataType::Int32, true))]; - - // Should be OK - missing fields will be filled with nulls - let result = validate_struct_compatibility(&source_fields, &target_fields); - assert!(result.is_ok()); - assert!(result.unwrap()); - } -} From 5be2fa78d51d3940e3ef0cb1c5ffb1cab22f6277 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Fri, 20 Jun 2025 14:50:00 +0800 Subject: [PATCH 56/57] fix: update code block syntax in documentation --- datafusion/common/src/nested_struct.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/common/src/nested_struct.rs b/datafusion/common/src/nested_struct.rs index d5c215da41ff4..f349b360f2385 100644 --- a/datafusion/common/src/nested_struct.rs +++ b/datafusion/common/src/nested_struct.rs @@ -145,7 +145,7 @@ pub fn cast_column(source_col: &ArrayRef, target_field: &Field) -> Result Date: Fri, 27 Jun 2025 17:50:56 +0800 Subject: [PATCH 57/57] Integrate nested schema support into DefaultSchemaAdapter extend DefaultSchemaAdapter to handle nested struct fields remove NestedStructSchemaAdapter module update re-exports and tests adjust nested stats mapping test --- datafusion/core/src/datasource/mod.rs | 1 - datafusion/datasource/src/mod.rs | 1 - .../src/nested_schema_adapter/adapter.rs | 94 --- .../src/nested_schema_adapter/factory.rs | 61 -- .../src/nested_schema_adapter/mod.rs | 26 - .../src/nested_schema_adapter/tests.rs | 533 ------------------ datafusion/datasource/src/schema_adapter.rs | 322 ++++++++++- 7 files changed, 301 insertions(+), 737 deletions(-) delete mode 100644 datafusion/datasource/src/nested_schema_adapter/adapter.rs delete mode 100644 datafusion/datasource/src/nested_schema_adapter/factory.rs delete mode 100644 datafusion/datasource/src/nested_schema_adapter/mod.rs delete mode 100644 datafusion/datasource/src/nested_schema_adapter/tests.rs diff --git a/datafusion/core/src/datasource/mod.rs b/datafusion/core/src/datasource/mod.rs index c7aef465b6ca4..94d651ddadd5c 100644 --- a/datafusion/core/src/datasource/mod.rs +++ b/datafusion/core/src/datasource/mod.rs @@ -42,7 +42,6 @@ pub use datafusion_catalog::default_table_source; pub use datafusion_catalog::memory; pub use datafusion_catalog::stream; pub use datafusion_catalog::view; -pub use datafusion_datasource::nested_schema_adapter; pub use datafusion_datasource::schema_adapter; pub use datafusion_datasource::sink; pub use datafusion_datasource::source; diff --git a/datafusion/datasource/src/mod.rs b/datafusion/datasource/src/mod.rs index c99e570ea1b84..c79efd11fcc5e 100644 --- a/datafusion/datasource/src/mod.rs +++ b/datafusion/datasource/src/mod.rs @@ -38,7 +38,6 @@ pub mod file_scan_config; pub mod file_sink_config; pub mod file_stream; pub mod memory; -pub mod nested_schema_adapter; pub mod schema_adapter; pub mod sink; pub mod source; diff --git a/datafusion/datasource/src/nested_schema_adapter/adapter.rs b/datafusion/datasource/src/nested_schema_adapter/adapter.rs deleted file mode 100644 index 0e1e2c441bb13..0000000000000 --- a/datafusion/datasource/src/nested_schema_adapter/adapter.rs +++ /dev/null @@ -1,94 +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 crate::schema_adapter::{ - create_field_mapping, SchemaAdapter, SchemaMapper, SchemaMapping, -}; -use arrow::{ - array::ArrayRef, - datatypes::{DataType::Struct, Field, Schema, SchemaRef}, -}; -use datafusion_common::nested_struct::{cast_column, validate_struct_compatibility}; -use datafusion_common::Result; -use std::sync::Arc; - -/// A SchemaAdapter that handles schema evolution for nested struct types -#[derive(Debug, Clone)] -pub struct NestedStructSchemaAdapter { - /// The schema for the table, projected to include only the fields being output (projected) by the - /// associated ParquetSource - projected_table_schema: SchemaRef, - /// The entire table schema for the table we're using this to adapt. - /// - /// This is used to evaluate any filters pushed down into the scan - /// which may refer to columns that are not referred to anywhere - /// else in the plan. - table_schema: SchemaRef, -} - -impl NestedStructSchemaAdapter { - /// Create a new NestedStructSchemaAdapter with the target schema - pub fn new(projected_table_schema: SchemaRef, table_schema: SchemaRef) -> Self { - Self { - projected_table_schema, - table_schema, - } - } - - pub fn projected_table_schema(&self) -> &Schema { - self.projected_table_schema.as_ref() - } - - pub fn table_schema(&self) -> &Schema { - self.table_schema.as_ref() - } -} - -impl SchemaAdapter for NestedStructSchemaAdapter { - fn map_column_index(&self, index: usize, file_schema: &Schema) -> Option { - let field_name = self.table_schema.field(index).name(); - file_schema.index_of(field_name).ok() - } - - fn map_schema( - &self, - file_schema: &Schema, - ) -> Result<(Arc, Vec)> { - let (field_mappings, projection) = create_field_mapping( - file_schema, - &self.projected_table_schema, - |file_field, table_field| { - // Special handling for struct fields - validate internal structure compatibility - match (file_field.data_type(), table_field.data_type()) { - (Struct(source_fields), Struct(target_fields)) => { - validate_struct_compatibility(source_fields, target_fields) - } - _ => crate::schema_adapter::can_cast_field(file_field, table_field), - } - }, - )?; - - Ok(( - Arc::new(SchemaMapping::new( - Arc::clone(&self.projected_table_schema), - field_mappings, - Arc::new(|array: &ArrayRef, field: &Field| cast_column(array, field)), - )), - projection, - )) - } -} diff --git a/datafusion/datasource/src/nested_schema_adapter/factory.rs b/datafusion/datasource/src/nested_schema_adapter/factory.rs deleted file mode 100644 index 520c5d73dfa15..0000000000000 --- a/datafusion/datasource/src/nested_schema_adapter/factory.rs +++ /dev/null @@ -1,61 +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 super::adapter::NestedStructSchemaAdapter; -use crate::schema_adapter::{SchemaAdapter, SchemaAdapterFactory}; -use arrow::datatypes::{DataType::Struct, Schema, SchemaRef}; -use std::sync::Arc; - -/// Factory for creating [`NestedStructSchemaAdapter`] -/// -/// This factory creates schema adapters that properly handle schema evolution -/// for nested struct fields, allowing new fields to be added to struct columns -/// over time. -#[derive(Debug, Clone, Default)] -pub struct NestedStructSchemaAdapterFactory; - -impl SchemaAdapterFactory for NestedStructSchemaAdapterFactory { - fn create( - &self, - projected_table_schema: SchemaRef, - table_schema: SchemaRef, - ) -> Box { - Box::new(NestedStructSchemaAdapter::new( - projected_table_schema, - table_schema, - )) - } -} - -impl NestedStructSchemaAdapterFactory { - /// Create a new factory for mapping batches from a file schema to a table - /// schema with support for nested struct evolution. - /// - /// This is a convenience method that handles nested struct fields properly. - pub fn from_schema(table_schema: SchemaRef) -> Box { - Self.create(Arc::clone(&table_schema), table_schema) - } - - /// Determines if a schema contains nested struct fields that would benefit - /// from special handling during schema evolution - pub fn has_nested_structs(schema: &Schema) -> bool { - schema - .fields() - .iter() - .any(|field| matches!(field.data_type(), Struct(_))) - } -} diff --git a/datafusion/datasource/src/nested_schema_adapter/mod.rs b/datafusion/datasource/src/nested_schema_adapter/mod.rs deleted file mode 100644 index 7494713337e05..0000000000000 --- a/datafusion/datasource/src/nested_schema_adapter/mod.rs +++ /dev/null @@ -1,26 +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. - -//! Nested schema adaptation utilities. - -pub mod adapter; -pub mod factory; -#[cfg(test)] -mod tests; - -pub use adapter::NestedStructSchemaAdapter; -pub use factory::NestedStructSchemaAdapterFactory; diff --git a/datafusion/datasource/src/nested_schema_adapter/tests.rs b/datafusion/datasource/src/nested_schema_adapter/tests.rs deleted file mode 100644 index e95e838048033..0000000000000 --- a/datafusion/datasource/src/nested_schema_adapter/tests.rs +++ /dev/null @@ -1,533 +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 crate::{ - nested_schema_adapter::{ - NestedStructSchemaAdapter, NestedStructSchemaAdapterFactory, - }, - schema_adapter::{DefaultSchemaAdapterFactory, SchemaAdapter, SchemaAdapterFactory}, -}; -use arrow::{ - array::{Array, StringBuilder, StructArray, TimestampMillisecondArray}, - compute::cast, - datatypes::{ - DataType::{Float64, Int16, Int32, Struct, Timestamp, Utf8}, - Field, Schema, SchemaRef, - TimeUnit::Millisecond, - }, - record_batch::RecordBatch, -}; -use datafusion_common::{ColumnStatistics, Result, ScalarValue}; -use std::sync::Arc; - -// ================================ -// Schema Evolution Tests -// ================================ - -#[test] -fn test_adapter_factory_selection() -> Result<()> { - // Test schemas for adapter selection logic - let simple_schema = Arc::new(Schema::new(vec![ - Field::new("id", Int32, false), - Field::new("name", Utf8, true), - Field::new("age", Int16, true), - ])); - - let nested_schema = Arc::new(Schema::new(vec![ - Field::new("id", Int32, false), - Field::new( - "metadata", - Struct( - vec![ - Field::new("created", Utf8, true), - Field::new("modified", Utf8, true), - ] - .into(), - ), - true, - ), - ])); - - // Source schema with missing field - let source_schema = Arc::new(Schema::new(vec![ - Field::new("id", Int32, false), - Field::new( - "metadata", - Struct( - vec![ - Field::new("created", Utf8, true), - // "modified" field is missing - ] - .into(), - ), - true, - ), - ])); - - // Test struct detection - assert!( - !NestedStructSchemaAdapterFactory::has_nested_structs(&simple_schema), - "Simple schema should not be detected as having nested structs" - ); - assert!( - NestedStructSchemaAdapterFactory::has_nested_structs(&nested_schema), - "Nested schema should be detected as having nested structs" - ); - - // Test adapter behavior with schema evolution - let default_adapter = - DefaultSchemaAdapterFactory.create(nested_schema.clone(), nested_schema.clone()); - let nested_adapter = NestedStructSchemaAdapterFactory - .create(nested_schema.clone(), nested_schema.clone()); - - // Default adapter should fail with schema evolution - assert!(default_adapter.map_schema(&source_schema).is_err()); - - // Nested adapter should handle schema evolution - assert!( - nested_adapter.map_schema(&source_schema).is_ok(), - "Nested adapter should handle schema with missing fields" - ); - - // Test factory selection logic - let adapter: Box = - if NestedStructSchemaAdapterFactory::has_nested_structs(&nested_schema) { - NestedStructSchemaAdapterFactory - .create(nested_schema.clone(), nested_schema.clone()) - } else { - DefaultSchemaAdapterFactory - .create(nested_schema.clone(), nested_schema.clone()) - }; - - assert!( - adapter.map_schema(&source_schema).is_ok(), - "Factory should select appropriate adapter that handles schema evolution" - ); - - Ok(()) -} - -#[test] -fn test_adapt_struct_with_added_nested_fields() -> Result<()> { - // Create test schemas - let (file_schema, table_schema) = create_test_schemas_with_nested_fields(); - - // Create batch with test data - let batch = create_test_batch_with_struct_data(&file_schema)?; - - // Create adapter and apply it - let mapped_batch = - adapt_batch_with_nested_schema_adapter(&file_schema, &table_schema, batch)?; - - // Verify the results - verify_adapted_batch_with_nested_fields(&mapped_batch, &table_schema)?; - - Ok(()) -} - -/// Create file and table schemas for testing nested field evolution -fn create_test_schemas_with_nested_fields() -> (SchemaRef, SchemaRef) { - // Create file schema with just location and timestamp_utc - let file_schema = Arc::new(Schema::new(vec![Field::new( - "info", - Struct( - vec![ - Field::new("location", Utf8, true), - Field::new( - "timestamp_utc", - Timestamp(Millisecond, Some("UTC".into())), - true, - ), - ] - .into(), - ), - true, - )])); - - // Create table schema with additional nested reason field - let table_schema = Arc::new(Schema::new(vec![Field::new( - "info", - Struct( - vec![ - Field::new("location", Utf8, true), - Field::new( - "timestamp_utc", - Timestamp(Millisecond, Some("UTC".into())), - true, - ), - Field::new( - "reason", - Struct( - vec![ - Field::new("_level", Float64, true), - Field::new( - "details", - Struct( - vec![ - Field::new("rurl", Utf8, true), - Field::new("s", Float64, true), - Field::new("t", Utf8, true), - ] - .into(), - ), - true, - ), - ] - .into(), - ), - true, - ), - ] - .into(), - ), - true, - )])); - - (file_schema, table_schema) -} - -/// Create a test RecordBatch with struct data matching the file schema -fn create_test_batch_with_struct_data(file_schema: &SchemaRef) -> Result { - let mut location_builder = StringBuilder::new(); - location_builder.append_value("San Francisco"); - location_builder.append_value("New York"); - - // Create timestamp array - let timestamp_array = TimestampMillisecondArray::from(vec![ - Some(1640995200000), // 2022-01-01 - Some(1641081600000), // 2022-01-02 - ]); - - // Create data type with UTC timezone to match the schema - let timestamp_type = Timestamp(Millisecond, Some("UTC".into())); - - // Cast the timestamp array to include the timezone metadata - let timestamp_array = cast(×tamp_array, ×tamp_type)?; - - let info_struct = StructArray::from(vec![ - ( - Arc::new(Field::new("location", Utf8, true)), - Arc::new(location_builder.finish()) as Arc, - ), - ( - Arc::new(Field::new("timestamp_utc", timestamp_type, true)), - timestamp_array, - ), - ]); - - Ok(RecordBatch::try_new( - Arc::clone(file_schema), - vec![Arc::new(info_struct)], - )?) -} - -/// Apply the nested schema adapter to the batch -fn adapt_batch_with_nested_schema_adapter( - file_schema: &SchemaRef, - table_schema: &SchemaRef, - batch: RecordBatch, -) -> Result { - let adapter = NestedStructSchemaAdapter::new( - Arc::clone(table_schema), - Arc::clone(table_schema), - ); - - let (mapper, _) = adapter.map_schema(file_schema.as_ref())?; - mapper.map_batch(batch) -} - -/// Verify the adapted batch has the expected structure and data -fn verify_adapted_batch_with_nested_fields( - mapped_batch: &RecordBatch, - table_schema: &SchemaRef, -) -> Result<()> { - // Verify the mapped batch structure and data - assert_eq!(mapped_batch.schema(), *table_schema); - assert_eq!(mapped_batch.num_rows(), 2); - - // Extract and verify the info struct column - let info_col = mapped_batch.column(0); - let info_array = info_col - .as_any() - .downcast_ref::() - .expect("Expected info column to be a StructArray"); - - // Verify the original fields are preserved - verify_preserved_fields(info_array)?; - - // Verify the reason field exists with correct structure - verify_reason_field_structure(info_array)?; - - Ok(()) -} - -/// Verify the original fields from file schema are preserved in the adapted batch -fn verify_preserved_fields(info_array: &StructArray) -> Result<()> { - // Verify location field - let location_col = info_array - .column_by_name("location") - .expect("Expected location field in struct"); - let location_array = location_col - .as_any() - .downcast_ref::() - .expect("Expected location to be a StringArray"); - - // Verify the location values are preserved - assert_eq!(location_array.value(0), "San Francisco"); - assert_eq!(location_array.value(1), "New York"); - - // Verify timestamp field - let timestamp_col = info_array - .column_by_name("timestamp_utc") - .expect("Expected timestamp_utc field in struct"); - let timestamp_array = timestamp_col - .as_any() - .downcast_ref::() - .expect("Expected timestamp_utc to be a TimestampMillisecondArray"); - - assert_eq!(timestamp_array.value(0), 1640995200000); - assert_eq!(timestamp_array.value(1), 1641081600000); - - Ok(()) -} - -/// Verify the added reason field structure and null values -fn verify_reason_field_structure(info_array: &StructArray) -> Result<()> { - // Verify the reason field exists and is null - let reason_col = info_array - .column_by_name("reason") - .expect("Expected reason field in struct"); - let reason_array = reason_col - .as_any() - .downcast_ref::() - .expect("Expected reason to be a StructArray"); - - // Verify reason has correct structure - assert_eq!(reason_array.fields().len(), 2); - assert!(reason_array.column_by_name("_level").is_some()); - assert!(reason_array.column_by_name("details").is_some()); - - // Verify details field has correct nested structure - let details_col = reason_array - .column_by_name("details") - .expect("Expected details field in reason struct"); - let details_array = details_col - .as_any() - .downcast_ref::() - .expect("Expected details to be a StructArray"); - - assert_eq!(details_array.fields().len(), 3); - assert!(details_array.column_by_name("rurl").is_some()); - assert!(details_array.column_by_name("s").is_some()); - assert!(details_array.column_by_name("t").is_some()); - - // Verify all added fields are null - for i in 0..2 { - assert!(reason_array.is_null(i), "reason field should be null"); - } - - Ok(()) -} - -// ================================ -// Data Mapping Tests -// ================================ - -// Helper function to verify column statistics match expected values -fn verify_column_statistics( - stats: &ColumnStatistics, - expected_null_count: Option, - expected_distinct_count: Option, - expected_min: Option, - expected_max: Option, - expected_sum: Option, -) { - if let Some(count) = expected_null_count { - assert_eq!( - stats.null_count, - datafusion_common::stats::Precision::Exact(count), - "Null count should match expected value" - ); - } - - if let Some(count) = expected_distinct_count { - assert_eq!( - stats.distinct_count, - datafusion_common::stats::Precision::Exact(count), - "Distinct count should match expected value" - ); - } - - if let Some(min) = expected_min { - assert_eq!( - stats.min_value, - datafusion_common::stats::Precision::Exact(min), - "Min value should match expected value" - ); - } - - if let Some(max) = expected_max { - assert_eq!( - stats.max_value, - datafusion_common::stats::Precision::Exact(max), - "Max value should match expected value" - ); - } - - if let Some(sum) = expected_sum { - assert_eq!( - stats.sum_value, - datafusion_common::stats::Precision::Exact(sum), - "Sum value should match expected value" - ); - } -} - -// Helper to create test column statistics -fn create_test_column_statistics( - null_count: usize, - distinct_count: usize, - min_value: Option, - max_value: Option, - sum_value: Option, -) -> ColumnStatistics { - ColumnStatistics { - null_count: datafusion_common::stats::Precision::Exact(null_count), - distinct_count: datafusion_common::stats::Precision::Exact(distinct_count), - min_value: min_value.map_or_else( - || datafusion_common::stats::Precision::Absent, - datafusion_common::stats::Precision::Exact, - ), - max_value: max_value.map_or_else( - || datafusion_common::stats::Precision::Absent, - datafusion_common::stats::Precision::Exact, - ), - sum_value: sum_value.map_or_else( - || datafusion_common::stats::Precision::Absent, - datafusion_common::stats::Precision::Exact, - ), - } -} - -#[test] -fn test_map_column_statistics() -> Result<()> { - // Create schemas with multiple columns - let file_schema = Arc::new(Schema::new(vec![ - Field::new("id", Int32, false), - Field::new( - "additionalInfo", - Struct( - vec![ - Field::new("location", Utf8, true), - Field::new( - "timestamp_utc", - Timestamp(Millisecond, Some("UTC".into())), - true, - ), - ] - .into(), - ), - true, - ), - ])); - - let table_schema = Arc::new(Schema::new(vec![ - Field::new("id", Int32, false), - Field::new( - "additionalInfo", - Struct( - vec![ - Field::new("location", Utf8, true), - Field::new( - "timestamp_utc", - Timestamp(Millisecond, Some("UTC".into())), - true, - ), - Field::new( - "reason", - Struct(vec![Field::new("_level", Float64, true)].into()), - true, - ), - ] - .into(), - ), - true, - ), - Field::new("status", Utf8, true), // Extra column in table schema - ])); - - // Create adapter and mapping - let adapter = NestedStructSchemaAdapter::new( - Arc::clone(&table_schema), - Arc::clone(&table_schema), - ); - - let (mapper, _) = adapter.map_schema(file_schema.as_ref())?; - - // Create file column statistics - let file_stats = vec![ - create_test_column_statistics( - 0, - 100, - Some(ScalarValue::Int32(Some(1))), - Some(ScalarValue::Int32(Some(100))), - Some(ScalarValue::Int32(Some(5100))), - ), - create_test_column_statistics(10, 50, None, None, None), - ]; - - // Map statistics - let table_stats = mapper.map_column_statistics(&file_stats)?; - - // Verify mapped statistics - assert_eq!( - table_stats.len(), - 3, - "Should have stats for all 3 columns in table schema" - ); - - // Verify ID column stats - verify_column_statistics( - &table_stats[0], - Some(0), - Some(100), - Some(ScalarValue::Int32(Some(1))), - Some(ScalarValue::Int32(Some(100))), - Some(ScalarValue::Int32(Some(5100))), - ); - - // Verify additionalInfo column stats - verify_column_statistics(&table_stats[1], Some(10), Some(50), None, None, None); - - // Verify status column has unknown stats - assert_eq!( - table_stats[2], - ColumnStatistics::new_unknown(), - "Missing column should have unknown statistics", - ); - - // Verify behavior when no statistics are provided - let empty_stats = vec![]; - let missing_stats = mapper.map_column_statistics(&empty_stats)?; - - assert_eq!(missing_stats.len(), 3); - for stat in missing_stats { - assert_eq!(stat, ColumnStatistics::new_unknown()); - } - - Ok(()) -} diff --git a/datafusion/datasource/src/schema_adapter.rs b/datafusion/datasource/src/schema_adapter.rs index c869f1ac19230..b43041c8d14db 100644 --- a/datafusion/datasource/src/schema_adapter.rs +++ b/datafusion/datasource/src/schema_adapter.rs @@ -22,10 +22,13 @@ //! can be stored external to a parquet file that maps parquet logical types to arrow types. use arrow::{ array::{new_null_array, ArrayRef, RecordBatch, RecordBatchOptions}, - compute::{can_cast_types, cast}, - datatypes::{Field, Schema, SchemaRef}, + compute::can_cast_types, + datatypes::{DataType, Field, Schema, SchemaRef}, +}; +use datafusion_common::{ + nested_struct::{cast_column, validate_struct_compatibility}, + plan_err, ColumnStatistics, }; -use datafusion_common::{plan_err, ColumnStatistics}; use std::{fmt::Debug, sync::Arc}; /// Function used by [`SchemaMapping`] to adapt a column from the file schema to /// the table schema. @@ -236,15 +239,22 @@ pub(crate) fn can_cast_field( file_field: &Field, table_field: &Field, ) -> datafusion_common::Result { - if can_cast_types(file_field.data_type(), table_field.data_type()) { - Ok(true) - } else { - plan_err!( - "Cannot cast file schema field {} of type {:?} to table schema field of type {:?}", - file_field.name(), - file_field.data_type(), - table_field.data_type() - ) + match (file_field.data_type(), table_field.data_type()) { + (DataType::Struct(source_fields), DataType::Struct(target_fields)) => { + validate_struct_compatibility(source_fields, target_fields) + } + _ => { + if can_cast_types(file_field.data_type(), table_field.data_type()) { + Ok(true) + } else { + plan_err!( + "Cannot cast file schema field {} of type {:?} to table schema field of type {:?}", + file_field.name(), + file_field.data_type(), + table_field.data_type() + ) + } + } } } @@ -281,9 +291,7 @@ impl SchemaAdapter for DefaultSchemaAdapter { Arc::new(SchemaMapping::new( Arc::clone(&self.projected_table_schema), field_mappings, - Arc::new(|array: &ArrayRef, field: &Field| { - Ok(cast(array, field.data_type())?) - }), + Arc::new(|array: &ArrayRef, field: &Field| cast_column(array, field)), )), projection, )) @@ -444,11 +452,12 @@ impl SchemaMapper for SchemaMapping { mod tests { use super::*; use arrow::{ - array::ArrayRef, + array::{Array, ArrayRef, StringBuilder, StructArray, TimestampMillisecondArray}, compute::cast, - datatypes::{DataType, Field}, + datatypes::{DataType, Field, TimeUnit}, + record_batch::RecordBatch, }; - use datafusion_common::{stats::Precision, Statistics}; + use datafusion_common::{stats::Precision, Result, ScalarValue, Statistics}; #[test] fn test_schema_mapping_map_statistics_basic() { @@ -622,9 +631,7 @@ mod tests { let mapping = SchemaMapping::new( Arc::clone(&projected_schema), field_mappings.clone(), - Arc::new(|array: &ArrayRef, field: &Field| { - Ok(cast(array, field.data_type())?) - }), + Arc::new(|array: &ArrayRef, field: &Field| cast_column(array, field)), ); // Check that fields were set correctly @@ -729,4 +736,277 @@ mod tests { assert_eq!(c_array.len(), 2); assert_eq!(c_array.null_count(), 2); } + + #[test] + fn test_adapt_struct_with_added_nested_fields() -> Result<()> { + let (file_schema, table_schema) = create_test_schemas_with_nested_fields(); + let batch = create_test_batch_with_struct_data(&file_schema)?; + + let adapter = DefaultSchemaAdapter { + projected_table_schema: Arc::clone(&table_schema), + }; + let (mapper, _) = adapter.map_schema(file_schema.as_ref())?; + let mapped_batch = mapper.map_batch(batch)?; + + verify_adapted_batch_with_nested_fields(&mapped_batch, &table_schema)?; + Ok(()) + } + + #[test] + fn test_map_column_statistics_struct() -> Result<()> { + let (file_schema, table_schema) = create_test_schemas_with_nested_fields(); + + let adapter = DefaultSchemaAdapter { + projected_table_schema: Arc::clone(&table_schema), + }; + let (mapper, _) = adapter.map_schema(file_schema.as_ref())?; + + let file_stats = vec![ + create_test_column_statistics( + 0, + 100, + Some(ScalarValue::Int32(Some(1))), + Some(ScalarValue::Int32(Some(100))), + Some(ScalarValue::Int32(Some(5100))), + ), + create_test_column_statistics(10, 50, None, None, None), + ]; + + let table_stats = mapper.map_column_statistics(&file_stats)?; + assert_eq!(table_stats.len(), 1); + verify_column_statistics( + &table_stats[0], + Some(0), + Some(100), + Some(ScalarValue::Int32(Some(1))), + Some(ScalarValue::Int32(Some(100))), + Some(ScalarValue::Int32(Some(5100))), + ); + let missing_stats = mapper.map_column_statistics(&[])?; + assert_eq!(missing_stats.len(), 1); + assert_eq!(missing_stats[0], ColumnStatistics::new_unknown()); + Ok(()) + } + + fn create_test_schemas_with_nested_fields() -> (SchemaRef, SchemaRef) { + let file_schema = Arc::new(Schema::new(vec![Field::new( + "info", + DataType::Struct( + vec![ + Field::new("location", DataType::Utf8, true), + Field::new( + "timestamp_utc", + DataType::Timestamp(TimeUnit::Millisecond, Some("UTC".into())), + true, + ), + ] + .into(), + ), + true, + )])); + + let table_schema = Arc::new(Schema::new(vec![Field::new( + "info", + DataType::Struct( + vec![ + Field::new("location", DataType::Utf8, true), + Field::new( + "timestamp_utc", + DataType::Timestamp(TimeUnit::Millisecond, Some("UTC".into())), + true, + ), + Field::new( + "reason", + DataType::Struct( + vec![ + Field::new("_level", DataType::Float64, true), + Field::new( + "details", + DataType::Struct( + vec![ + Field::new("rurl", DataType::Utf8, true), + Field::new("s", DataType::Float64, true), + Field::new("t", DataType::Utf8, true), + ] + .into(), + ), + true, + ), + ] + .into(), + ), + true, + ), + ] + .into(), + ), + true, + )])); + + (file_schema, table_schema) + } + + fn create_test_batch_with_struct_data( + file_schema: &SchemaRef, + ) -> Result { + let mut location_builder = StringBuilder::new(); + location_builder.append_value("San Francisco"); + location_builder.append_value("New York"); + + let timestamp_array = TimestampMillisecondArray::from(vec![ + Some(1640995200000), + Some(1641081600000), + ]); + + let timestamp_type = + DataType::Timestamp(TimeUnit::Millisecond, Some("UTC".into())); + let timestamp_array = cast(×tamp_array, ×tamp_type)?; + + let info_struct = StructArray::from(vec![ + ( + Arc::new(Field::new("location", DataType::Utf8, true)), + Arc::new(location_builder.finish()) as ArrayRef, + ), + ( + Arc::new(Field::new("timestamp_utc", timestamp_type, true)), + timestamp_array, + ), + ]); + + Ok(RecordBatch::try_new( + Arc::clone(file_schema), + vec![Arc::new(info_struct)], + )?) + } + + fn verify_adapted_batch_with_nested_fields( + mapped_batch: &RecordBatch, + table_schema: &SchemaRef, + ) -> Result<()> { + assert_eq!(mapped_batch.schema(), *table_schema); + assert_eq!(mapped_batch.num_rows(), 2); + + let info_col = mapped_batch.column(0); + let info_array = info_col + .as_any() + .downcast_ref::() + .expect("Expected info column to be a StructArray"); + + verify_preserved_fields(info_array)?; + verify_reason_field_structure(info_array)?; + Ok(()) + } + + fn verify_preserved_fields(info_array: &StructArray) -> Result<()> { + let location_col = info_array + .column_by_name("location") + .expect("Expected location field in struct"); + let location_array = location_col + .as_any() + .downcast_ref::() + .expect("Expected location to be a StringArray"); + assert_eq!(location_array.value(0), "San Francisco"); + assert_eq!(location_array.value(1), "New York"); + + let timestamp_col = info_array + .column_by_name("timestamp_utc") + .expect("Expected timestamp_utc field in struct"); + let timestamp_array = timestamp_col + .as_any() + .downcast_ref::() + .expect("Expected timestamp_utc to be a TimestampMillisecondArray"); + assert_eq!(timestamp_array.value(0), 1640995200000); + assert_eq!(timestamp_array.value(1), 1641081600000); + Ok(()) + } + + fn verify_reason_field_structure(info_array: &StructArray) -> Result<()> { + let reason_col = info_array + .column_by_name("reason") + .expect("Expected reason field in struct"); + let reason_array = reason_col + .as_any() + .downcast_ref::() + .expect("Expected reason to be a StructArray"); + assert_eq!(reason_array.fields().len(), 2); + assert!(reason_array.column_by_name("_level").is_some()); + assert!(reason_array.column_by_name("details").is_some()); + + let details_col = reason_array + .column_by_name("details") + .expect("Expected details field in reason struct"); + let details_array = details_col + .as_any() + .downcast_ref::() + .expect("Expected details to be a StructArray"); + assert_eq!(details_array.fields().len(), 3); + assert!(details_array.column_by_name("rurl").is_some()); + assert!(details_array.column_by_name("s").is_some()); + assert!(details_array.column_by_name("t").is_some()); + for i in 0..2 { + assert!(reason_array.is_null(i), "reason field should be null"); + } + Ok(()) + } + + fn verify_column_statistics( + stats: &ColumnStatistics, + expected_null_count: Option, + expected_distinct_count: Option, + expected_min: Option, + expected_max: Option, + expected_sum: Option, + ) { + if let Some(count) = expected_null_count { + assert_eq!( + stats.null_count, + Precision::Exact(count), + "Null count should match expected value" + ); + } + if let Some(count) = expected_distinct_count { + assert_eq!( + stats.distinct_count, + Precision::Exact(count), + "Distinct count should match expected value" + ); + } + if let Some(min) = expected_min { + assert_eq!( + stats.min_value, + Precision::Exact(min), + "Min value should match expected value" + ); + } + if let Some(max) = expected_max { + assert_eq!( + stats.max_value, + Precision::Exact(max), + "Max value should match expected value" + ); + } + if let Some(sum) = expected_sum { + assert_eq!( + stats.sum_value, + Precision::Exact(sum), + "Sum value should match expected value" + ); + } + } + + fn create_test_column_statistics( + null_count: usize, + distinct_count: usize, + min_value: Option, + max_value: Option, + sum_value: Option, + ) -> ColumnStatistics { + ColumnStatistics { + null_count: Precision::Exact(null_count), + distinct_count: Precision::Exact(distinct_count), + min_value: min_value.map_or_else(|| Precision::Absent, Precision::Exact), + max_value: max_value.map_or_else(|| Precision::Absent, Precision::Exact), + sum_value: sum_value.map_or_else(|| Precision::Absent, Precision::Exact), + } + } }