From 2d958325c223fb38266644455b95ec421fac2905 Mon Sep 17 00:00:00 2001 From: Subham Singhal Date: Sun, 15 Feb 2026 21:47:29 +0530 Subject: [PATCH 01/15] Adds support for parquet field id --- datafusion/common/src/config.rs | 27 +- .../common/src/file_options/parquet_writer.rs | 1 + datafusion/core/tests/parquet/field_id.rs | 525 ++++++++++++++++++ datafusion/core/tests/parquet/mod.rs | 1 + .../datasource-parquet/src/file_format.rs | 34 +- datafusion/datasource-parquet/src/metadata.rs | 70 ++- datafusion/datasource-parquet/src/opener.rs | 24 +- datafusion/datasource-parquet/src/source.rs | 3 + datafusion/physical-expr/src/utils/mod.rs | 382 +++++++++++++ 9 files changed, 1039 insertions(+), 28 deletions(-) create mode 100644 datafusion/core/tests/parquet/field_id.rs diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index dad12c1c6bc91..49d0bf72081d9 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -751,6 +751,11 @@ config_namespace! { /// parquet reader setting. 0 means no caching. pub max_predicate_cache_size: Option, default = None + /// (reading) If true, use Parquet field IDs for column resolution instead of + /// column names. This enables schema evolution with renamed/reordered columns. + /// When field IDs are unavailable, falls back to name-based matching. + pub field_id_read_enabled: bool, default = false + // The following options affect writing to parquet files // and map to parquet::file::properties::WriterProperties @@ -1142,12 +1147,6 @@ config_namespace! { /// /// Default: true pub enable_sort_pushdown: bool, default = true - - /// When set to true, the optimizer will extract leaf expressions - /// (such as `get_field`) from filter/sort/join nodes into projections - /// closer to the leaf table scans, and push those projections down - /// towards the leaf nodes. - pub enable_leaf_expression_pushdown: bool, default = true } } @@ -3071,22 +3070,6 @@ config_namespace! { /// If not specified, the default level for the compression algorithm is used. pub compression_level: Option, default = None pub schema_infer_max_rec: Option, default = None - /// The JSON format to use when reading files. - /// - /// When `true` (default), expects newline-delimited JSON (NDJSON): - /// ```text - /// {"key1": 1, "key2": "val"} - /// {"key1": 2, "key2": "vals"} - /// ``` - /// - /// When `false`, expects JSON array format: - /// ```text - /// [ - /// {"key1": 1, "key2": "val"}, - /// {"key1": 2, "key2": "vals"} - /// ] - /// ``` - pub newline_delimited: bool, default = true } } diff --git a/datafusion/common/src/file_options/parquet_writer.rs b/datafusion/common/src/file_options/parquet_writer.rs index f6608d16c1022..5af4d9a7ac613 100644 --- a/datafusion/common/src/file_options/parquet_writer.rs +++ b/datafusion/common/src/file_options/parquet_writer.rs @@ -209,6 +209,7 @@ impl ParquetOptions { coerce_int96: _, // not used for writer props skip_arrow_metadata: _, max_predicate_cache_size: _, + field_id_read_enabled: _, // not used for writer props } = self; let mut builder = WriterProperties::builder() diff --git a/datafusion/core/tests/parquet/field_id.rs b/datafusion/core/tests/parquet/field_id.rs new file mode 100644 index 0000000000000..d96e1009196a1 --- /dev/null +++ b/datafusion/core/tests/parquet/field_id.rs @@ -0,0 +1,525 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! Integration tests for Parquet field ID support + +use arrow::array::{ + Array, Int32Array, Int64Array, RecordBatch, StringArray, StringViewArray +}; +use arrow::datatypes::{DataType, Field, Schema}; +use datafusion::prelude::*; +use datafusion_common::Result; +use parquet::arrow::ArrowWriter; +use parquet::file::properties::WriterProperties; +use std::collections::HashMap; +use std::fs::File; +use std::sync::Arc; +use tempfile::TempDir; + +/// Helper to create a test Parquet file with field IDs +fn create_parquet_file_with_field_ids( + path: &str, + schema: Arc, + batches: Vec, +) -> Result<()> { + let file = File::create(path)?; + let props = WriterProperties::builder().build(); + let mut writer = ArrowWriter::try_new(file, schema, Some(props))?; + + for batch in batches { + writer.write(&batch)?; + } + + writer.close()?; + Ok(()) +} + +/// Helper to create a schema with field IDs in metadata +fn schema_with_field_ids(fields: Vec<(String, DataType, i32)>) -> Schema { + let fields_with_ids: Vec = fields + .into_iter() + .map(|(name, dtype, field_id)| { + let mut metadata = HashMap::new(); + metadata.insert("PARQUET:field_id".to_string(), field_id.to_string()); + Field::new(name, dtype, false).with_metadata(metadata) + }) + .collect(); + + Schema::new(fields_with_ids) +} + +#[tokio::test] +async fn test_read_parquet_with_field_ids_enabled() -> Result<()> { + let tmp_dir = TempDir::new()?; + let file_path = tmp_dir.path().join("test.parquet"); + + // Create schema with field IDs + let schema = Arc::new(schema_with_field_ids(vec![ + ("user_id".to_string(), DataType::Int64, 1), + ("amount".to_string(), DataType::Int64, 2), + ("name".to_string(), DataType::Utf8, 3), + ])); + + // Create test data + let batch = RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(Int64Array::from(vec![1, 2, 3])), + Arc::new(Int64Array::from(vec![100, 200, 300])), + Arc::new(StringArray::from(vec!["Alice", "Bob", "Charlie"])), + ], + )?; + + create_parquet_file_with_field_ids(file_path.to_str().unwrap(), schema, vec![batch])?; + + // Create context with field ID reading enabled + let ctx = SessionContext::new(); + ctx.sql("SET datafusion.execution.parquet.field_id_read_enabled = true") + .await? + .collect() + .await?; + + // Register table and query + ctx.register_parquet( + "test", + file_path.to_str().unwrap(), + ParquetReadOptions::default() + ) + .await?; + + let df = ctx.sql("SELECT user_id, amount, name FROM test").await?; + let results = df.collect().await?; + + // Verify results + assert_eq!(results.len(), 1); + assert_eq!(results[0].num_rows(), 3); + + let user_ids = results[0] + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(user_ids.value(0), 1); + assert_eq!(user_ids.value(1), 2); + assert_eq!(user_ids.value(2), 3); + + Ok(()) +} + +#[tokio::test] +async fn test_read_parquet_with_field_ids_disabled() -> Result<()> { + let tmp_dir = TempDir::new()?; + let file_path = tmp_dir.path().join("test.parquet"); + + // Create schema with field IDs + let schema = Arc::new(schema_with_field_ids(vec![ + ("user_id".to_string(), DataType::Int64, 1), + ("amount".to_string(), DataType::Int64, 2), + ])); + + let batch = RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(Int64Array::from(vec![1, 2, 3])), + Arc::new(Int64Array::from(vec![100, 200, 300])), + ], + )?; + + create_parquet_file_with_field_ids(file_path.to_str().unwrap(), schema, vec![batch])?; + + // Create context with field ID reading disabled (default) + let ctx = SessionContext::new(); + + ctx.register_parquet("test", + file_path.to_str().unwrap(), + ParquetReadOptions::default() + ) + .await?; + + let df = ctx.sql("SELECT user_id, amount FROM test").await?; + let results = df.collect().await?; + + // Should still work with name-based matching + assert_eq!(results.len(), 1); + assert_eq!(results[0].num_rows(), 3); + + Ok(()) +} + +#[tokio::test] +async fn test_schema_evolution_renamed_columns() -> Result<()> { + let tmp_dir = TempDir::new()?; + let file_path = tmp_dir.path().join("test.parquet"); + + // Write file with original column names and field IDs + let write_schema = Arc::new(schema_with_field_ids(vec![ + ("user_id".to_string(), DataType::Int64, 1), + ("amount".to_string(), DataType::Int64, 2), + ])); + + let batch = RecordBatch::try_new( + Arc::clone(&write_schema), + vec![ + Arc::new(Int64Array::from(vec![101, 102, 103])), + Arc::new(Int64Array::from(vec![500, 600, 700])), + ], + )?; + + create_parquet_file_with_field_ids( + file_path.to_str().unwrap(), + write_schema, + vec![batch], + )?; + + // Create context with field ID reading enabled + let ctx = SessionContext::new(); + ctx.sql("SET datafusion.execution.parquet.field_id_read_enabled = true") + .await? + .collect() + .await?; + + // Register table with original names + ctx.register_parquet("test", + file_path.to_str().unwrap(), + ParquetReadOptions::default() + ) + .await?; + + // Query should work with original names + let df = ctx.sql("SELECT user_id, amount FROM test").await?; + let results = df.collect().await?; + + assert_eq!(results.len(), 1); + let user_ids = results[0] + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(user_ids.value(0), 101); + + Ok(()) +} + +#[tokio::test] +async fn test_schema_evolution_reordered_columns() -> Result<()> { + let tmp_dir = TempDir::new()?; + let file_path = tmp_dir.path().join("test.parquet"); + + // Write file with columns in order: a, b, c + let write_schema = Arc::new(schema_with_field_ids(vec![ + ("a".to_string(), DataType::Int32, 1), + ("b".to_string(), DataType::Int32, 2), + ("c".to_string(), DataType::Int32, 3), + ])); + + let batch = RecordBatch::try_new( + Arc::clone(&write_schema), + vec![ + Arc::new(Int32Array::from(vec![10, 20, 30])), + Arc::new(Int32Array::from(vec![40, 50, 60])), + Arc::new(Int32Array::from(vec![70, 80, 90])), + ], + )?; + + create_parquet_file_with_field_ids( + file_path.to_str().unwrap(), + write_schema, + vec![batch], + )?; + + // Create context with field ID reading enabled + let ctx = SessionContext::new(); + ctx.sql("SET datafusion.execution.parquet.field_id_read_enabled = true") + .await? + .collect() + .await?; + + ctx.register_parquet("test", + file_path.to_str().unwrap(), + ParquetReadOptions::default() + ) + .await?; + + // Query columns in different order: c, a, b + let df = ctx.sql("SELECT c, a, b FROM test").await?; + let results = df.collect().await?; + + assert_eq!(results.len(), 1); + + let c_vals = results[0] + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + let a_vals = results[0] + .column(1) + .as_any() + .downcast_ref::() + .unwrap(); + let b_vals = results[0] + .column(2) + .as_any() + .downcast_ref::() + .unwrap(); + + // Verify correct data regardless of order + assert_eq!(c_vals.value(0), 70); + assert_eq!(a_vals.value(0), 10); + assert_eq!(b_vals.value(0), 40); + + Ok(()) +} + +#[tokio::test] +async fn test_projection_with_field_ids() -> Result<()> { + let tmp_dir = TempDir::new()?; + let file_path = tmp_dir.path().join("test.parquet"); + + // Create schema with field IDs + let schema = Arc::new(schema_with_field_ids(vec![ + ("a".to_string(), DataType::Int32, 1), + ("b".to_string(), DataType::Int32, 2), + ("c".to_string(), DataType::Int32, 3), + ("d".to_string(), DataType::Int32, 4), + ])); + + let batch = RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(Int32Array::from(vec![1, 2, 3])), + Arc::new(Int32Array::from(vec![10, 20, 30])), + Arc::new(Int32Array::from(vec![100, 200, 300])), + Arc::new(Int32Array::from(vec![1000, 2000, 3000])), + ], + )?; + + create_parquet_file_with_field_ids(file_path.to_str().unwrap(), schema, vec![batch])?; + + let ctx = SessionContext::new(); + ctx.sql("SET datafusion.execution.parquet.field_id_read_enabled = true") + .await? + .collect() + .await?; + + ctx.register_parquet("test", + file_path.to_str().unwrap(), + ParquetReadOptions::default() + ).await?; + + // Project only columns a and c + let df = ctx.sql("SELECT a, c FROM test").await?; + let results = df.collect().await?; + + assert_eq!(results.len(), 1); + assert_eq!(results[0].num_columns(), 2); + + let a_vals = results[0] + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + let c_vals = results[0] + .column(1) + .as_any() + .downcast_ref::() + .unwrap(); + + assert_eq!(a_vals.value(0), 1); + assert_eq!(c_vals.value(0), 100); + + Ok(()) +} + +#[tokio::test] +async fn test_filter_with_field_ids() -> Result<()> { + let tmp_dir = TempDir::new()?; + let file_path = tmp_dir.path().join("test.parquet"); + + let schema = Arc::new(schema_with_field_ids(vec![ + ("id".to_string(), DataType::Int32, 1), + ("value".to_string(), DataType::Int32, 2), + ])); + + let batch = RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(Int32Array::from(vec![1, 2, 3, 4, 5])), + Arc::new(Int32Array::from(vec![10, 20, 30, 40, 50])), + ], + )?; + + create_parquet_file_with_field_ids(file_path.to_str().unwrap(), schema, vec![batch])?; + + let ctx = SessionContext::new(); + ctx.sql("SET datafusion.execution.parquet.field_id_read_enabled = true") + .await? + .collect() + .await?; + + ctx.register_parquet("test", + file_path.to_str().unwrap(), + ParquetReadOptions::default() + ) + .await?; + + // Filter with field IDs + let df = ctx.sql("SELECT id, value FROM test WHERE value > 25").await?; + let results = df.collect().await?; + + assert_eq!(results.len(), 1); + assert_eq!(results[0].num_rows(), 3); // Should have rows with values 30, 40, 50 + + let id_vals = results[0] + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(id_vals.value(0), 3); + assert_eq!(id_vals.value(1), 4); + assert_eq!(id_vals.value(2), 5); + + Ok(()) +} + +#[tokio::test] +async fn test_aggregation_with_field_ids() -> Result<()> { + let tmp_dir = TempDir::new()?; + let file_path = tmp_dir.path().join("test.parquet"); + + let schema = Arc::new(schema_with_field_ids(vec![ + ("category".to_string(), DataType::Utf8, 1), + ("value".to_string(), DataType::Int32, 2), + ])); + + let batch = RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(StringArray::from(vec!["A", "B", "A", "B", "A"])), + Arc::new(Int32Array::from(vec![10, 20, 30, 40, 50])), + ], + )?; + + create_parquet_file_with_field_ids(file_path.to_str().unwrap(), schema, vec![batch])?; + + let ctx = SessionContext::new(); + ctx.sql("SET datafusion.execution.parquet.field_id_read_enabled = true") + .await? + .collect() + .await?; + + ctx.register_parquet( + "test", + file_path.to_str().unwrap(), + ParquetReadOptions::default() + ) + .await?; + + // Aggregate with field IDs + let df = ctx + .sql("SELECT category, SUM(value) as total FROM test GROUP BY category ORDER BY category") + .await?; + let results = df.collect().await?; + + assert_eq!(results.len(), 1); + assert_eq!(results[0].num_rows(), 2); + + // Get category column - it might be StringArray or StringViewArray depending on config + let category_col = results[0].column(0); + let categories: Vec<&str> = match category_col.data_type() { + DataType::Utf8 => { + category_col + .as_any() + .downcast_ref::() + .unwrap() + .iter() + .map(|v| v.unwrap()) + .collect() + } + DataType::Utf8View => { + category_col + .as_any() + .downcast_ref::() + .unwrap() + .iter() + .map(|v| v.unwrap()) + .collect() + } + _ => panic!("Unexpected data type for category column: {:?}", category_col.data_type()), + }; + + let totals = results[0] + .column(1) + .as_any() + .downcast_ref::() + .unwrap(); + + assert_eq!(categories[0], "A"); + assert_eq!(totals.value(0), 90); // 10 + 30 + 50 + + assert_eq!(categories[1], "B"); + assert_eq!(totals.value(1), 60); // 20 + 40 + + Ok(()) +} + +#[tokio::test] +async fn test_fallback_to_name_when_no_field_ids() -> Result<()> { + let tmp_dir = TempDir::new()?; + let file_path = tmp_dir.path().join("test.parquet"); + + // Create schema WITHOUT field IDs in metadata + let schema = Arc::new(Schema::new(vec![ + Field::new("user_id", DataType::Int64, false), + Field::new("amount", DataType::Int64, false), + ])); + + let batch = RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(Int64Array::from(vec![1, 2, 3])), + Arc::new(Int64Array::from(vec![100, 200, 300])), + ], + )?; + + create_parquet_file_with_field_ids(file_path.to_str().unwrap(), schema, vec![batch])?; + + // Create context with field ID reading enabled + let ctx = SessionContext::new(); + ctx.sql("SET datafusion.execution.parquet.field_id_read_enabled = true") + .await? + .collect() + .await?; + + ctx.register_parquet("test", file_path.to_str().unwrap(), ParquetReadOptions::default()) + .await?; + + // Should fall back to name-based matching + let df = ctx.sql("SELECT user_id, amount FROM test").await?; + let results = df.collect().await?; + + assert_eq!(results.len(), 1); + assert_eq!(results[0].num_rows(), 3); + + let user_ids = results[0] + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(user_ids.value(0), 1); + + Ok(()) +} diff --git a/datafusion/core/tests/parquet/mod.rs b/datafusion/core/tests/parquet/mod.rs index 5a05718936509..df99c24629716 100644 --- a/datafusion/core/tests/parquet/mod.rs +++ b/datafusion/core/tests/parquet/mod.rs @@ -49,6 +49,7 @@ mod custom_reader; mod encryption; mod expr_adapter; mod external_access_plan; +mod field_id; mod file_statistics; mod filter_pushdown; mod ordering; diff --git a/datafusion/datasource-parquet/src/file_format.rs b/datafusion/datasource-parquet/src/file_format.rs index d59b42ed15d15..32b63b9603089 100644 --- a/datafusion/datasource-parquet/src/file_format.rs +++ b/datafusion/datasource-parquet/src/file_format.rs @@ -633,6 +633,7 @@ impl ParquetFormat { pub fn apply_file_schema_type_coercions( table_schema: &Schema, file_schema: &Schema, + enable_field_ids: bool, ) -> Option { let mut needs_view_transform = false; let mut needs_string_transform = false; @@ -660,6 +661,22 @@ pub fn apply_file_schema_type_coercions( }) .collect(); + // Build field ID to field mapping if field IDs are enabled + let table_field_by_id: HashMap> = if enable_field_ids { + table_schema + .fields() + .iter() + .filter_map(|f| { + f.metadata() + .get("PARQUET:field_id") + .and_then(|id_str| id_str.parse::().ok()) + .map(|id| (id, f)) + }) + .collect() + } else { + HashMap::new() + }; + // Early return if no transformation needed if !needs_view_transform && !needs_string_transform { return None; @@ -672,8 +689,23 @@ pub fn apply_file_schema_type_coercions( let field_name = field.name(); let field_type = field.data_type(); + // Try to find matching table field by field ID or name + let table_type = if enable_field_ids { + // Try field ID matching first + field + .metadata() + .get("PARQUET:field_id") + .and_then(|id_str| id_str.parse::().ok()) + .and_then(|id| table_field_by_id.get(&id)) + .map(|f| f.data_type()) + .or_else(|| table_fields.get(field_name).copied()) + } else { + // Name-based matching only + table_fields.get(field_name).copied() + }; + // Look up the corresponding field type in the table schema - if let Some(table_type) = table_fields.get(field_name) { + if let Some(table_type) = table_type { match (table_type, field_type) { // table schema uses string type, coerce the file schema to use string type ( diff --git a/datafusion/datasource-parquet/src/metadata.rs b/datafusion/datasource-parquet/src/metadata.rs index 5a4c0bcdd514d..502964dcbe3be 100644 --- a/datafusion/datasource-parquet/src/metadata.rs +++ b/datafusion/datasource-parquet/src/metadata.rs @@ -68,6 +68,55 @@ pub struct DFParquetMetadata<'a> { file_metadata_cache: Option>, /// timeunit to coerce INT96 timestamps to pub coerce_int96: Option, + /// Whether to extract and use Parquet field IDs for column resolution + pub enable_field_ids: bool, +} + +/// Extracts Parquet field IDs and stores them in Arrow field metadata +/// under the key "PARQUET:field_id" +/// +/// # Limitations +/// +/// TODO: Currently only supports flat schemas (top-level primitive fields). +/// Nested field IDs within structs, lists, and maps are not yet supported. +/// This requires recursive traversal of the Parquet schema tree to extract +/// field IDs at all nesting levels. See PARQUET_FIELD_ID_IMPLEMENTATION.md +/// for details on nested schema support. +fn add_field_ids_to_arrow_schema( + arrow_schema: Schema, + parquet_schema: &SchemaDescriptor, +) -> Result { + use arrow::datatypes::Field; + + let fields_with_ids: Vec> = arrow_schema + .fields() + .iter() + .enumerate() + .map(|(idx, field)| { + // Get the corresponding Parquet column descriptor + // TODO: This only works for flat schemas - parquet_schema.column(idx) + // returns leaf columns only, missing nested struct fields + let col_desc = parquet_schema.column(idx); + + // Extract field ID from the schema type + // Field IDs are optional in Parquet; if not set, they may be 0 or negative + let field_id = col_desc.self_type().get_basic_info().id(); + + if field_id > 0 { + // Add field ID to field metadata + let mut metadata = field.metadata().clone(); + metadata.insert("PARQUET:field_id".to_string(), field_id.to_string()); + Arc::new(field.as_ref().clone().with_metadata(metadata)) + } else { + Arc::clone(field) + } + }) + .collect(); + + Ok(Schema::new_with_metadata( + fields_with_ids, + arrow_schema.metadata().clone(), + )) } impl<'a> DFParquetMetadata<'a> { @@ -79,6 +128,7 @@ impl<'a> DFParquetMetadata<'a> { decryption_properties: None, file_metadata_cache: None, coerce_int96: None, + enable_field_ids: false, } } @@ -112,6 +162,12 @@ impl<'a> DFParquetMetadata<'a> { self } + /// Set whether to extract and use Parquet field IDs for column resolution + pub fn with_enable_field_ids(mut self, enable: bool) -> Self { + self.enable_field_ids = enable; + self + } + /// Fetch parquet metadata from the remote object store pub async fn fetch_metadata(&self) -> Result> { let Self { @@ -121,6 +177,7 @@ impl<'a> DFParquetMetadata<'a> { decryption_properties, file_metadata_cache, coerce_int96: _, + enable_field_ids: _, } = self; let fetch = ObjectStoreFetch::new(*store, object_meta); @@ -180,10 +237,17 @@ impl<'a> DFParquetMetadata<'a> { let metadata = self.fetch_metadata().await?; let file_metadata = metadata.file_metadata(); - let schema = parquet_to_arrow_schema( + let mut schema = parquet_to_arrow_schema( file_metadata.schema_descr(), file_metadata.key_value_metadata(), )?; + + // Add field IDs if requested + if self.enable_field_ids { + schema = add_field_ids_to_arrow_schema(schema, file_metadata.schema_descr())?; + } + + // Apply INT96 coercion if configured let schema = self .coerce_int96 .as_ref() @@ -195,6 +259,7 @@ impl<'a> DFParquetMetadata<'a> { ) }) .unwrap_or(schema); + Ok(schema) } @@ -279,8 +344,9 @@ impl<'a> DFParquetMetadata<'a> { file_metadata.key_value_metadata(), )?; + // Apply type coercions without field ID matching (statistics use name-based matching) if let Some(merged) = - apply_file_schema_type_coercions(logical_file_schema, &physical_file_schema) + apply_file_schema_type_coercions(logical_file_schema, &physical_file_schema, false) { physical_file_schema = merged; } diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index f87a30265a17b..d2a7da98539ca 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -27,7 +27,9 @@ use arrow::array::{RecordBatch, RecordBatchOptions}; use arrow::datatypes::DataType; use datafusion_datasource::file_stream::{FileOpenFuture, FileOpener}; use datafusion_physical_expr::projection::ProjectionExprs; -use datafusion_physical_expr::utils::reassign_expr_columns; +use datafusion_physical_expr::utils::{ + reassign_expr_columns, reassign_expr_columns_with_field_ids, +}; use datafusion_physical_expr_adapter::replace_columns_with_literals; use std::collections::HashMap; use std::pin::Pin; @@ -120,6 +122,8 @@ pub(super) struct ParquetOpener { pub max_predicate_cache_size: Option, /// Whether to read row groups in reverse order pub reverse_row_groups: bool, + /// Whether to use Parquet field IDs for column resolution + pub field_id_read_enabled: bool, } /// Represents a prepared access plan with optional row selection @@ -205,6 +209,7 @@ impl FileOpener for ParquetOpener { )?; let batch_size = self.batch_size; + let field_id_read_enabled = self.field_id_read_enabled; // Calculate the output schema from the original projection (before literal replacement) // so we get correct field names from column references @@ -378,6 +383,7 @@ impl FileOpener for ParquetOpener { if let Some(merged) = apply_file_schema_type_coercions( &logical_file_schema, &physical_file_schema, + field_id_read_enabled, ) { physical_file_schema = Arc::new(merged); options = options.with_schema(Arc::clone(&physical_file_schema)); @@ -624,8 +630,17 @@ impl FileOpener for ParquetOpener { // Rebase column indices to match the narrowed stream schema. // The projection expressions have indices based on physical_file_schema, // but the stream only contains the columns selected by the ProjectionMask. - let projection = projection - .try_map_exprs(|expr| reassign_expr_columns(expr, &stream_schema))?; + let projection = if field_id_read_enabled { + projection.try_map_exprs(|expr| { + reassign_expr_columns_with_field_ids( + expr, + &physical_file_schema, + &stream_schema, + ) + })? + } else { + projection.try_map_exprs(|expr| reassign_expr_columns(expr, &stream_schema))? + }; let projector = projection.make_projector(&stream_schema)?; @@ -1064,6 +1079,7 @@ mod test { max_predicate_cache_size: Option, reverse_row_groups: bool, preserve_order: bool, + field_id_read_enabled: bool, } impl ParquetOpenerBuilder { @@ -1090,6 +1106,7 @@ mod test { max_predicate_cache_size: None, reverse_row_groups: false, preserve_order: false, + field_id_read_enabled: false, } } @@ -1197,6 +1214,7 @@ mod test { encryption_factory: None, max_predicate_cache_size: self.max_predicate_cache_size, reverse_row_groups: self.reverse_row_groups, + field_id_read_enabled: self.field_id_read_enabled, preserve_order: self.preserve_order, } } diff --git a/datafusion/datasource-parquet/src/source.rs b/datafusion/datasource-parquet/src/source.rs index 75d87a4cd16fc..0cac3ed4c2fb7 100644 --- a/datafusion/datasource-parquet/src/source.rs +++ b/datafusion/datasource-parquet/src/source.rs @@ -541,6 +541,8 @@ impl FileSource for ParquetSource { .as_ref() .map(|time_unit| parse_coerce_int96_string(time_unit.as_str()).unwrap()); + let field_id_read_enabled = self.table_parquet_options.global.field_id_read_enabled; + let opener = Arc::new(ParquetOpener { partition_index: partition, projection: self.projection.clone(), @@ -568,6 +570,7 @@ impl FileSource for ParquetSource { encryption_factory: self.get_encryption_factory_with_config(), max_predicate_cache_size: self.max_predicate_cache_size(), reverse_row_groups: self.reverse_row_groups, + field_id_read_enabled, }); Ok(opener) } diff --git a/datafusion/physical-expr/src/utils/mod.rs b/datafusion/physical-expr/src/utils/mod.rs index 2cdc326f5dd36..0ebdfde23cc7d 100644 --- a/datafusion/physical-expr/src/utils/mod.rs +++ b/datafusion/physical-expr/src/utils/mod.rs @@ -265,9 +265,95 @@ pub fn reassign_expr_columns( .data() } +/// Extract Parquet field ID from Arrow field metadata +fn get_field_id(field: &arrow::datatypes::Field) -> Option { + field + .metadata() + .get("PARQUET:field_id") + .and_then(|s| s.parse::().ok()) +} + +/// Find field index by field ID with fallback to name-based matching +/// +/// # Limitations +/// +/// TODO: Currently only supports flat schemas. For nested schemas, this function +/// would need to accept a field path (e.g., ["address", "city"]) and return +/// a path of indices. This requires matching nested field IDs at each level +/// of the schema hierarchy. +fn find_field_index( + column_name: &str, + source_schema: &Schema, + target_schema: &Schema, +) -> Result { + // Try to find the field in source schema + let source_field = source_schema.field_with_name(column_name)?; + + // Check if field has a field ID + if let Some(source_field_id) = get_field_id(source_field) { + // Search target schema for matching field ID + // TODO: For nested schemas, this needs to recursively match field IDs + // through the struct hierarchy + for (idx, target_field) in target_schema.fields().iter().enumerate() { + if let Some(target_field_id) = get_field_id(target_field) { + if source_field_id == target_field_id { + return Ok(idx); + } + } + } + } + + // Fallback to name-based matching + Ok(target_schema.index_of(column_name)?) +} + +/// Re-assign column indices in expressions using field ID-based matching. +/// +/// This function traverses the expression tree and updates all `Column` references +/// to use field IDs for matching between source and target schemas, falling back +/// to name-based matching when field IDs are unavailable. +/// +/// # Arguments +/// +/// * `expr` - The physical expression to update +/// * `source_schema` - The schema that the expression currently references +/// * `target_schema` - The schema to map columns to +/// +/// # Limitations +/// +/// TODO: Currently only supports flat schemas (top-level columns). Nested field +/// references (e.g., "address.city") are not yet supported. Supporting nested +/// fields would require: +/// - Path-based field ID matching through struct hierarchies +/// - Recursive traversal of both expression tree and schema tree +/// - Updates to Column representation to track nested paths +/// +/// # Errors +/// +/// This function will return an error if any column in the expression cannot be found +/// in the target schema by either field ID or name. +pub fn reassign_expr_columns_with_field_ids( + expr: Arc, + source_schema: &Schema, + target_schema: &Schema, +) -> Result> { + expr.transform_down(|expr| { + if let Some(column) = expr.as_any().downcast_ref::() { + let index = find_field_index(column.name(), source_schema, target_schema)?; + return Ok(Transformed::yes(Arc::new(Column::new( + column.name(), + index, + )))); + } + Ok(Transformed::no(expr)) + }) + .data() +} + #[cfg(test)] pub(crate) mod tests { use std::any::Any; + use std::collections::HashMap; use std::fmt::{Display, Formatter}; use super::*; @@ -562,4 +648,300 @@ pub(crate) mod tests { assert_eq!(collect_columns(&expr3), expected); Ok(()) } + + // ======================================================================== + // Field ID Tests + // ======================================================================== + + #[test] + fn test_get_field_id_present() { + let mut metadata = HashMap::new(); + metadata.insert("PARQUET:field_id".to_string(), "42".to_string()); + let field = Field::new("test", DataType::Int64, false).with_metadata(metadata); + + assert_eq!(get_field_id(&field), Some(42)); + } + + #[test] + fn test_get_field_id_absent() { + let field = Field::new("test", DataType::Int64, false); + assert_eq!(get_field_id(&field), None); + } + + #[test] + fn test_get_field_id_invalid() { + let mut metadata = HashMap::new(); + metadata.insert("PARQUET:field_id".to_string(), "not_a_number".to_string()); + let field = Field::new("test", DataType::Int64, false).with_metadata(metadata); + + assert_eq!(get_field_id(&field), None); + } + + #[test] + fn test_find_field_index_by_field_id() -> Result<()> { + // Source schema: field IDs present + let mut metadata1 = HashMap::new(); + metadata1.insert("PARQUET:field_id".to_string(), "1".to_string()); + let mut metadata2 = HashMap::new(); + metadata2.insert("PARQUET:field_id".to_string(), "2".to_string()); + + let source_schema = Schema::new(vec![ + Field::new("user_id", DataType::Int64, false).with_metadata(metadata1.clone()), + Field::new("amount", DataType::Float64, false).with_metadata(metadata2.clone()), + ]); + + // Target schema: renamed columns but same field IDs + let target_schema = Schema::new(vec![ + Field::new("customer_id", DataType::Int64, false).with_metadata(metadata1), + Field::new("price", DataType::Float64, false).with_metadata(metadata2), + ]); + + // Should match by field ID, not name + let index = find_field_index("user_id", &source_schema, &target_schema)?; + assert_eq!(index, 0, "user_id (field_id=1) should match customer_id at index 0"); + + let index = find_field_index("amount", &source_schema, &target_schema)?; + assert_eq!(index, 1, "amount (field_id=2) should match price at index 1"); + + Ok(()) + } + + #[test] + fn test_find_field_index_by_field_id_reordered() -> Result<()> { + // Source schema: columns in order [a, b, c] + let mut meta_a = HashMap::new(); + meta_a.insert("PARQUET:field_id".to_string(), "1".to_string()); + let mut meta_b = HashMap::new(); + meta_b.insert("PARQUET:field_id".to_string(), "2".to_string()); + let mut meta_c = HashMap::new(); + meta_c.insert("PARQUET:field_id".to_string(), "3".to_string()); + + let source_schema = Schema::new(vec![ + Field::new("a", DataType::Int64, false).with_metadata(meta_a.clone()), + Field::new("b", DataType::Int64, false).with_metadata(meta_b.clone()), + Field::new("c", DataType::Int64, false).with_metadata(meta_c.clone()), + ]); + + // Target schema: columns reordered [c, a, b] + let target_schema = Schema::new(vec![ + Field::new("c", DataType::Int64, false).with_metadata(meta_c), + Field::new("a", DataType::Int64, false).with_metadata(meta_a), + Field::new("b", DataType::Int64, false).with_metadata(meta_b), + ]); + + // Should match by field ID + assert_eq!(find_field_index("a", &source_schema, &target_schema)?, 1); + assert_eq!(find_field_index("b", &source_schema, &target_schema)?, 2); + assert_eq!(find_field_index("c", &source_schema, &target_schema)?, 0); + + Ok(()) + } + + #[test] + fn test_find_field_index_fallback_to_name() -> Result<()> { + // Source schema: no field IDs + let source_schema = Schema::new(vec![ + Field::new("user_id", DataType::Int64, false), + Field::new("amount", DataType::Float64, false), + ]); + + // Target schema: no field IDs + let target_schema = Schema::new(vec![ + Field::new("user_id", DataType::Int64, false), + Field::new("amount", DataType::Float64, false), + ]); + + // Should fall back to name-based matching + assert_eq!(find_field_index("user_id", &source_schema, &target_schema)?, 0); + assert_eq!(find_field_index("amount", &source_schema, &target_schema)?, 1); + + Ok(()) + } + + #[test] + fn test_find_field_index_mixed_field_ids() -> Result<()> { + + // Source schema: some fields have IDs, some don't + let mut metadata1 = HashMap::new(); + metadata1.insert("PARQUET:field_id".to_string(), "1".to_string()); + + let source_schema = Schema::new(vec![ + Field::new("a", DataType::Int64, false).with_metadata(metadata1.clone()), + Field::new("b", DataType::Int64, false), // No field ID + ]); + + let target_schema = Schema::new(vec![ + Field::new("renamed_a", DataType::Int64, false).with_metadata(metadata1), + Field::new("b", DataType::Int64, false), + ]); + + // Field with ID should match by ID + assert_eq!(find_field_index("a", &source_schema, &target_schema)?, 0); + + // Field without ID should match by name + assert_eq!(find_field_index("b", &source_schema, &target_schema)?, 1); + + Ok(()) + } + + #[test] + fn test_find_field_index_not_found() { + let source_schema = Schema::new(vec![ + Field::new("a", DataType::Int64, false), + ]); + + let target_schema = Schema::new(vec![ + Field::new("b", DataType::Int64, false), + ]); + + // Should fail to find non-existent field + let result = find_field_index("a", &source_schema, &target_schema); + assert!(result.is_err()); + } + + #[test] + fn test_reassign_expr_columns_with_field_ids_simple() -> Result<()> { + + // Source schema: full file schema + let mut meta1 = HashMap::new(); + meta1.insert("PARQUET:field_id".to_string(), "1".to_string()); + let mut meta2 = HashMap::new(); + meta2.insert("PARQUET:field_id".to_string(), "2".to_string()); + let mut meta3 = HashMap::new(); + meta3.insert("PARQUET:field_id".to_string(), "3".to_string()); + + let source_schema = Schema::new(vec![ + Field::new("user_id", DataType::Int64, false).with_metadata(meta1.clone()), + Field::new("name", DataType::Utf8, false).with_metadata(meta2), + Field::new("age", DataType::Int32, false).with_metadata(meta3.clone()), + ]); + + // Target schema: projected schema (only user_id and age) + let target_schema = Schema::new(vec![ + Field::new("user_id", DataType::Int64, false).with_metadata(meta1), + Field::new("age", DataType::Int32, false).with_metadata(meta3), + ]); + + // Expression references age at index 2 in source schema + let expr: Arc = Arc::new(Column::new("age", 2)); + + // After transformation, should reference age at index 1 in target schema + let result = reassign_expr_columns_with_field_ids(expr, &source_schema, &target_schema)?; + + let column = result.as_any().downcast_ref::().unwrap(); + assert_eq!(column.name(), "age"); + assert_eq!(column.index(), 1, "age should be at index 1 in target schema"); + + Ok(()) + } + + #[test] + fn test_reassign_expr_columns_with_field_ids_complex() -> Result<()> { + + // Source schema + let mut meta1 = HashMap::new(); + meta1.insert("PARQUET:field_id".to_string(), "1".to_string()); + let mut meta2 = HashMap::new(); + meta2.insert("PARQUET:field_id".to_string(), "2".to_string()); + let mut meta3 = HashMap::new(); + meta3.insert("PARQUET:field_id".to_string(), "3".to_string()); + + let source_schema = Schema::new(vec![ + Field::new("a", DataType::Int32, false).with_metadata(meta1.clone()), + Field::new("b", DataType::Int32, false).with_metadata(meta2.clone()), + Field::new("c", DataType::Int32, false).with_metadata(meta3.clone()), + ]); + + // Target schema: only columns a and c (b excluded) + let target_schema = Schema::new(vec![ + Field::new("a", DataType::Int32, false).with_metadata(meta1), + Field::new("c", DataType::Int32, false).with_metadata(meta3), + ]); + + // Expression: a@0 + c@2 + let expr = binary( + col("a", &source_schema)?, + Operator::Plus, + col("c", &source_schema)?, + &source_schema, + )?; + + // After transformation: a@0 + c@1 + let result = reassign_expr_columns_with_field_ids(expr, &source_schema, &target_schema)?; + + // Verify it's still a binary expression + let binary_expr = result.as_any().downcast_ref::().unwrap(); + + // Check left side (a) + let left_col = binary_expr.left().as_any().downcast_ref::().unwrap(); + assert_eq!(left_col.name(), "a"); + assert_eq!(left_col.index(), 0); + + // Check right side (c) + let right_col = binary_expr.right().as_any().downcast_ref::().unwrap(); + assert_eq!(right_col.name(), "c"); + assert_eq!(right_col.index(), 1, "c should be remapped from index 2 to 1"); + + Ok(()) + } + + #[test] + fn test_reassign_expr_columns_with_field_ids_renamed_columns() -> Result<()> { + + // Source schema (file schema with old names) + let mut meta1 = HashMap::new(); + meta1.insert("PARQUET:field_id".to_string(), "1".to_string()); + let mut meta2 = HashMap::new(); + meta2.insert("PARQUET:field_id".to_string(), "2".to_string()); + + let source_schema = Schema::new(vec![ + Field::new("user_id", DataType::Int64, false).with_metadata(meta1.clone()), + Field::new("amount", DataType::Float64, false).with_metadata(meta2.clone()), + ]); + + // Target schema (query schema with renamed columns) + let target_schema = Schema::new(vec![ + Field::new("customer_id", DataType::Int64, false).with_metadata(meta1), + Field::new("price", DataType::Float64, false).with_metadata(meta2), + ]); + + // Expression references old names at their source indices + let expr: Arc = Arc::new(Column::new("user_id", 0)); + + // After transformation, should still reference by old name but correct index + let result = reassign_expr_columns_with_field_ids(expr, &source_schema, &target_schema)?; + + let column = result.as_any().downcast_ref::().unwrap(); + assert_eq!(column.name(), "user_id", "Name should remain user_id"); + assert_eq!(column.index(), 0, "Should match customer_id at index 0 via field_id"); + + Ok(()) + } + + #[test] + fn test_reassign_expr_columns_with_field_ids_no_field_ids() -> Result<()> { + // Schemas without field IDs - should fall back to name matching + let source_schema = Schema::new(vec![ + Field::new("a", DataType::Int32, false), + Field::new("b", DataType::Int32, false), + Field::new("c", DataType::Int32, false), + ]); + + let target_schema = Schema::new(vec![ + Field::new("a", DataType::Int32, false), + Field::new("c", DataType::Int32, false), + ]); + + // Expression: c@2 + let expr: Arc = Arc::new(Column::new("c", 2)); + + // Should fall back to name-based matching + let result = reassign_expr_columns_with_field_ids(expr, &source_schema, &target_schema)?; + + let column = result.as_any().downcast_ref::().unwrap(); + assert_eq!(column.name(), "c"); + assert_eq!(column.index(), 1, "c should be found by name at index 1"); + + Ok(()) + } } From 5db09e88e3dc9e448dbfe81f35adce0dfd8501ce Mon Sep 17 00:00:00 2001 From: Subham Singhal Date: Sun, 15 Feb 2026 22:06:13 +0530 Subject: [PATCH 02/15] Fix lint --- datafusion/core/tests/parquet/field_id.rs | 105 ++++++++++-------- datafusion/datasource-parquet/src/metadata.rs | 8 +- datafusion/datasource-parquet/src/opener.rs | 3 +- datafusion/datasource-parquet/src/source.rs | 3 +- datafusion/physical-expr/src/utils/mod.rs | 80 ++++++++----- 5 files changed, 122 insertions(+), 77 deletions(-) diff --git a/datafusion/core/tests/parquet/field_id.rs b/datafusion/core/tests/parquet/field_id.rs index d96e1009196a1..f85a1447f512b 100644 --- a/datafusion/core/tests/parquet/field_id.rs +++ b/datafusion/core/tests/parquet/field_id.rs @@ -18,7 +18,7 @@ //! Integration tests for Parquet field ID support use arrow::array::{ - Array, Int32Array, Int64Array, RecordBatch, StringArray, StringViewArray + Array, Int32Array, Int64Array, RecordBatch, StringArray, StringViewArray, }; use arrow::datatypes::{DataType, Field, Schema}; use datafusion::prelude::*; @@ -97,9 +97,9 @@ async fn test_read_parquet_with_field_ids_enabled() -> Result<()> { ctx.register_parquet( "test", file_path.to_str().unwrap(), - ParquetReadOptions::default() + ParquetReadOptions::default(), ) - .await?; + .await?; let df = ctx.sql("SELECT user_id, amount, name FROM test").await?; let results = df.collect().await?; @@ -144,11 +144,12 @@ async fn test_read_parquet_with_field_ids_disabled() -> Result<()> { // Create context with field ID reading disabled (default) let ctx = SessionContext::new(); - ctx.register_parquet("test", - file_path.to_str().unwrap(), - ParquetReadOptions::default() + ctx.register_parquet( + "test", + file_path.to_str().unwrap(), + ParquetReadOptions::default(), ) - .await?; + .await?; let df = ctx.sql("SELECT user_id, amount FROM test").await?; let results = df.collect().await?; @@ -193,11 +194,12 @@ async fn test_schema_evolution_renamed_columns() -> Result<()> { .await?; // Register table with original names - ctx.register_parquet("test", - file_path.to_str().unwrap(), - ParquetReadOptions::default() + ctx.register_parquet( + "test", + file_path.to_str().unwrap(), + ParquetReadOptions::default(), ) - .await?; + .await?; // Query should work with original names let df = ctx.sql("SELECT user_id, amount FROM test").await?; @@ -248,11 +250,12 @@ async fn test_schema_evolution_reordered_columns() -> Result<()> { .collect() .await?; - ctx.register_parquet("test", - file_path.to_str().unwrap(), - ParquetReadOptions::default() + ctx.register_parquet( + "test", + file_path.to_str().unwrap(), + ParquetReadOptions::default(), ) - .await?; + .await?; // Query columns in different order: c, a, b let df = ctx.sql("SELECT c, a, b FROM test").await?; @@ -315,10 +318,12 @@ async fn test_projection_with_field_ids() -> Result<()> { .collect() .await?; - ctx.register_parquet("test", - file_path.to_str().unwrap(), - ParquetReadOptions::default() - ).await?; + ctx.register_parquet( + "test", + file_path.to_str().unwrap(), + ParquetReadOptions::default(), + ) + .await?; // Project only columns a and c let df = ctx.sql("SELECT a, c FROM test").await?; @@ -370,14 +375,17 @@ async fn test_filter_with_field_ids() -> Result<()> { .collect() .await?; - ctx.register_parquet("test", - file_path.to_str().unwrap(), - ParquetReadOptions::default() + ctx.register_parquet( + "test", + file_path.to_str().unwrap(), + ParquetReadOptions::default(), ) - .await?; + .await?; // Filter with field IDs - let df = ctx.sql("SELECT id, value FROM test WHERE value > 25").await?; + let df = ctx + .sql("SELECT id, value FROM test WHERE value > 25") + .await?; let results = df.collect().await?; assert_eq!(results.len(), 1); @@ -424,9 +432,9 @@ async fn test_aggregation_with_field_ids() -> Result<()> { ctx.register_parquet( "test", file_path.to_str().unwrap(), - ParquetReadOptions::default() + ParquetReadOptions::default(), ) - .await?; + .await?; // Aggregate with field IDs let df = ctx @@ -440,25 +448,24 @@ async fn test_aggregation_with_field_ids() -> Result<()> { // Get category column - it might be StringArray or StringViewArray depending on config let category_col = results[0].column(0); let categories: Vec<&str> = match category_col.data_type() { - DataType::Utf8 => { - category_col - .as_any() - .downcast_ref::() - .unwrap() - .iter() - .map(|v| v.unwrap()) - .collect() - } - DataType::Utf8View => { - category_col - .as_any() - .downcast_ref::() - .unwrap() - .iter() - .map(|v| v.unwrap()) - .collect() - } - _ => panic!("Unexpected data type for category column: {:?}", category_col.data_type()), + DataType::Utf8 => category_col + .as_any() + .downcast_ref::() + .unwrap() + .iter() + .map(|v| v.unwrap()) + .collect(), + DataType::Utf8View => category_col + .as_any() + .downcast_ref::() + .unwrap() + .iter() + .map(|v| v.unwrap()) + .collect(), + _ => panic!( + "Unexpected data type for category column: {:?}", + category_col.data_type() + ), }; let totals = results[0] @@ -504,8 +511,12 @@ async fn test_fallback_to_name_when_no_field_ids() -> Result<()> { .collect() .await?; - ctx.register_parquet("test", file_path.to_str().unwrap(), ParquetReadOptions::default()) - .await?; + ctx.register_parquet( + "test", + file_path.to_str().unwrap(), + ParquetReadOptions::default(), + ) + .await?; // Should fall back to name-based matching let df = ctx.sql("SELECT user_id, amount FROM test").await?; diff --git a/datafusion/datasource-parquet/src/metadata.rs b/datafusion/datasource-parquet/src/metadata.rs index 502964dcbe3be..74a83e17263e2 100644 --- a/datafusion/datasource-parquet/src/metadata.rs +++ b/datafusion/datasource-parquet/src/metadata.rs @@ -345,9 +345,11 @@ impl<'a> DFParquetMetadata<'a> { )?; // Apply type coercions without field ID matching (statistics use name-based matching) - if let Some(merged) = - apply_file_schema_type_coercions(logical_file_schema, &physical_file_schema, false) - { + if let Some(merged) = apply_file_schema_type_coercions( + logical_file_schema, + &physical_file_schema, + false, + ) { physical_file_schema = merged; } diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index d2a7da98539ca..16bbde344ca95 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -639,7 +639,8 @@ impl FileOpener for ParquetOpener { ) })? } else { - projection.try_map_exprs(|expr| reassign_expr_columns(expr, &stream_schema))? + projection + .try_map_exprs(|expr| reassign_expr_columns(expr, &stream_schema))? }; let projector = projection.make_projector(&stream_schema)?; diff --git a/datafusion/datasource-parquet/src/source.rs b/datafusion/datasource-parquet/src/source.rs index 0cac3ed4c2fb7..b6e6a10bb27e3 100644 --- a/datafusion/datasource-parquet/src/source.rs +++ b/datafusion/datasource-parquet/src/source.rs @@ -541,7 +541,8 @@ impl FileSource for ParquetSource { .as_ref() .map(|time_unit| parse_coerce_int96_string(time_unit.as_str()).unwrap()); - let field_id_read_enabled = self.table_parquet_options.global.field_id_read_enabled; + let field_id_read_enabled = + self.table_parquet_options.global.field_id_read_enabled; let opener = Arc::new(ParquetOpener { partition_index: partition, diff --git a/datafusion/physical-expr/src/utils/mod.rs b/datafusion/physical-expr/src/utils/mod.rs index 0ebdfde23cc7d..aee12c4ee1cdd 100644 --- a/datafusion/physical-expr/src/utils/mod.rs +++ b/datafusion/physical-expr/src/utils/mod.rs @@ -686,8 +686,10 @@ pub(crate) mod tests { metadata2.insert("PARQUET:field_id".to_string(), "2".to_string()); let source_schema = Schema::new(vec![ - Field::new("user_id", DataType::Int64, false).with_metadata(metadata1.clone()), - Field::new("amount", DataType::Float64, false).with_metadata(metadata2.clone()), + Field::new("user_id", DataType::Int64, false) + .with_metadata(metadata1.clone()), + Field::new("amount", DataType::Float64, false) + .with_metadata(metadata2.clone()), ]); // Target schema: renamed columns but same field IDs @@ -698,10 +700,16 @@ pub(crate) mod tests { // Should match by field ID, not name let index = find_field_index("user_id", &source_schema, &target_schema)?; - assert_eq!(index, 0, "user_id (field_id=1) should match customer_id at index 0"); + assert_eq!( + index, 0, + "user_id (field_id=1) should match customer_id at index 0" + ); let index = find_field_index("amount", &source_schema, &target_schema)?; - assert_eq!(index, 1, "amount (field_id=2) should match price at index 1"); + assert_eq!( + index, 1, + "amount (field_id=2) should match price at index 1" + ); Ok(()) } @@ -752,15 +760,20 @@ pub(crate) mod tests { ]); // Should fall back to name-based matching - assert_eq!(find_field_index("user_id", &source_schema, &target_schema)?, 0); - assert_eq!(find_field_index("amount", &source_schema, &target_schema)?, 1); + assert_eq!( + find_field_index("user_id", &source_schema, &target_schema)?, + 0 + ); + assert_eq!( + find_field_index("amount", &source_schema, &target_schema)?, + 1 + ); Ok(()) } #[test] fn test_find_field_index_mixed_field_ids() -> Result<()> { - // Source schema: some fields have IDs, some don't let mut metadata1 = HashMap::new(); metadata1.insert("PARQUET:field_id".to_string(), "1".to_string()); @@ -786,13 +799,9 @@ pub(crate) mod tests { #[test] fn test_find_field_index_not_found() { - let source_schema = Schema::new(vec![ - Field::new("a", DataType::Int64, false), - ]); + let source_schema = Schema::new(vec![Field::new("a", DataType::Int64, false)]); - let target_schema = Schema::new(vec![ - Field::new("b", DataType::Int64, false), - ]); + let target_schema = Schema::new(vec![Field::new("b", DataType::Int64, false)]); // Should fail to find non-existent field let result = find_field_index("a", &source_schema, &target_schema); @@ -801,7 +810,6 @@ pub(crate) mod tests { #[test] fn test_reassign_expr_columns_with_field_ids_simple() -> Result<()> { - // Source schema: full file schema let mut meta1 = HashMap::new(); meta1.insert("PARQUET:field_id".to_string(), "1".to_string()); @@ -826,18 +834,22 @@ pub(crate) mod tests { let expr: Arc = Arc::new(Column::new("age", 2)); // After transformation, should reference age at index 1 in target schema - let result = reassign_expr_columns_with_field_ids(expr, &source_schema, &target_schema)?; + let result = + reassign_expr_columns_with_field_ids(expr, &source_schema, &target_schema)?; let column = result.as_any().downcast_ref::().unwrap(); assert_eq!(column.name(), "age"); - assert_eq!(column.index(), 1, "age should be at index 1 in target schema"); + assert_eq!( + column.index(), + 1, + "age should be at index 1 in target schema" + ); Ok(()) } #[test] fn test_reassign_expr_columns_with_field_ids_complex() -> Result<()> { - // Source schema let mut meta1 = HashMap::new(); meta1.insert("PARQUET:field_id".to_string(), "1".to_string()); @@ -867,27 +879,39 @@ pub(crate) mod tests { )?; // After transformation: a@0 + c@1 - let result = reassign_expr_columns_with_field_ids(expr, &source_schema, &target_schema)?; + let result = + reassign_expr_columns_with_field_ids(expr, &source_schema, &target_schema)?; // Verify it's still a binary expression let binary_expr = result.as_any().downcast_ref::().unwrap(); // Check left side (a) - let left_col = binary_expr.left().as_any().downcast_ref::().unwrap(); + let left_col = binary_expr + .left() + .as_any() + .downcast_ref::() + .unwrap(); assert_eq!(left_col.name(), "a"); assert_eq!(left_col.index(), 0); // Check right side (c) - let right_col = binary_expr.right().as_any().downcast_ref::().unwrap(); + let right_col = binary_expr + .right() + .as_any() + .downcast_ref::() + .unwrap(); assert_eq!(right_col.name(), "c"); - assert_eq!(right_col.index(), 1, "c should be remapped from index 2 to 1"); + assert_eq!( + right_col.index(), + 1, + "c should be remapped from index 2 to 1" + ); Ok(()) } #[test] fn test_reassign_expr_columns_with_field_ids_renamed_columns() -> Result<()> { - // Source schema (file schema with old names) let mut meta1 = HashMap::new(); meta1.insert("PARQUET:field_id".to_string(), "1".to_string()); @@ -909,11 +933,16 @@ pub(crate) mod tests { let expr: Arc = Arc::new(Column::new("user_id", 0)); // After transformation, should still reference by old name but correct index - let result = reassign_expr_columns_with_field_ids(expr, &source_schema, &target_schema)?; + let result = + reassign_expr_columns_with_field_ids(expr, &source_schema, &target_schema)?; let column = result.as_any().downcast_ref::().unwrap(); assert_eq!(column.name(), "user_id", "Name should remain user_id"); - assert_eq!(column.index(), 0, "Should match customer_id at index 0 via field_id"); + assert_eq!( + column.index(), + 0, + "Should match customer_id at index 0 via field_id" + ); Ok(()) } @@ -936,7 +965,8 @@ pub(crate) mod tests { let expr: Arc = Arc::new(Column::new("c", 2)); // Should fall back to name-based matching - let result = reassign_expr_columns_with_field_ids(expr, &source_schema, &target_schema)?; + let result = + reassign_expr_columns_with_field_ids(expr, &source_schema, &target_schema)?; let column = result.as_any().downcast_ref::().unwrap(); assert_eq!(column.name(), "c"); From cf2315c8190cbaf18b1136485ad0edef24288302 Mon Sep 17 00:00:00 2001 From: Subham Singhal Date: Sun, 15 Feb 2026 22:42:06 +0530 Subject: [PATCH 03/15] Fix build failure --- datafusion/common/src/config.rs | 22 +++++++++++++++++++ .../common/src/file_options/parquet_writer.rs | 2 ++ datafusion/datasource-parquet/src/metadata.rs | 5 +++-- datafusion/physical-expr/src/utils/mod.rs | 8 +++---- datafusion/proto-common/src/from_proto/mod.rs | 1 + .../proto/src/logical_plan/file_formats.rs | 1 + 6 files changed, 33 insertions(+), 6 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 49d0bf72081d9..86956dfe523fd 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -1147,6 +1147,12 @@ config_namespace! { /// /// Default: true pub enable_sort_pushdown: bool, default = true + + /// When set to true, the optimizer will extract leaf expressions + /// (such as `get_field`) from filter/sort/join nodes into projections + /// closer to the leaf table scans, and push those projections down + /// towards the leaf nodes. + pub enable_leaf_expression_pushdown: bool, default = true } } @@ -3070,6 +3076,22 @@ config_namespace! { /// If not specified, the default level for the compression algorithm is used. pub compression_level: Option, default = None pub schema_infer_max_rec: Option, default = None + /// The JSON format to use when reading files. + /// + /// When `true` (default), expects newline-delimited JSON (NDJSON): + /// ```text + /// {"key1": 1, "key2": "val"} + /// {"key1": 2, "key2": "vals"} + /// ``` + /// + /// When `false`, expects JSON array format: + /// ```text + /// [ + /// {"key1": 1, "key2": "val"}, + /// {"key1": 2, "key2": "vals"} + /// ] + /// ``` + pub newline_delimited: bool, default = true } } diff --git a/datafusion/common/src/file_options/parquet_writer.rs b/datafusion/common/src/file_options/parquet_writer.rs index 5af4d9a7ac613..85a281caa3ac3 100644 --- a/datafusion/common/src/file_options/parquet_writer.rs +++ b/datafusion/common/src/file_options/parquet_writer.rs @@ -461,6 +461,7 @@ mod tests { skip_arrow_metadata: defaults.skip_arrow_metadata, coerce_int96: None, max_predicate_cache_size: defaults.max_predicate_cache_size, + field_id_read_enabled: defaults.field_id_read_enabled, } } @@ -575,6 +576,7 @@ mod tests { binary_as_string: global_options_defaults.binary_as_string, skip_arrow_metadata: global_options_defaults.skip_arrow_metadata, coerce_int96: None, + field_id_read_enabled: global_options_defaults.field_id_read_enabled, }, column_specific_options, key_value_metadata, diff --git a/datafusion/datasource-parquet/src/metadata.rs b/datafusion/datasource-parquet/src/metadata.rs index 74a83e17263e2..b79827c2f9f49 100644 --- a/datafusion/datasource-parquet/src/metadata.rs +++ b/datafusion/datasource-parquet/src/metadata.rs @@ -83,7 +83,7 @@ pub struct DFParquetMetadata<'a> { /// field IDs at all nesting levels. See PARQUET_FIELD_ID_IMPLEMENTATION.md /// for details on nested schema support. fn add_field_ids_to_arrow_schema( - arrow_schema: Schema, + arrow_schema: &Schema, parquet_schema: &SchemaDescriptor, ) -> Result { use arrow::datatypes::Field; @@ -244,7 +244,8 @@ impl<'a> DFParquetMetadata<'a> { // Add field IDs if requested if self.enable_field_ids { - schema = add_field_ids_to_arrow_schema(schema, file_metadata.schema_descr())?; + schema = + add_field_ids_to_arrow_schema(&schema, file_metadata.schema_descr())?; } // Apply INT96 coercion if configured diff --git a/datafusion/physical-expr/src/utils/mod.rs b/datafusion/physical-expr/src/utils/mod.rs index aee12c4ee1cdd..54155e7fbb2e4 100644 --- a/datafusion/physical-expr/src/utils/mod.rs +++ b/datafusion/physical-expr/src/utils/mod.rs @@ -295,10 +295,10 @@ fn find_field_index( // TODO: For nested schemas, this needs to recursively match field IDs // through the struct hierarchy for (idx, target_field) in target_schema.fields().iter().enumerate() { - if let Some(target_field_id) = get_field_id(target_field) { - if source_field_id == target_field_id { - return Ok(idx); - } + if let Some(target_field_id) = get_field_id(target_field) + && source_field_id == target_field_id + { + return Ok(idx); } } } diff --git a/datafusion/proto-common/src/from_proto/mod.rs b/datafusion/proto-common/src/from_proto/mod.rs index ca8a269958d73..294b6c1231e46 100644 --- a/datafusion/proto-common/src/from_proto/mod.rs +++ b/datafusion/proto-common/src/from_proto/mod.rs @@ -1090,6 +1090,7 @@ impl TryFrom<&protobuf::ParquetOptions> for ParquetOptions { max_predicate_cache_size: value.max_predicate_cache_size_opt.map(|opt| match opt { protobuf::parquet_options::MaxPredicateCacheSizeOpt::MaxPredicateCacheSize(v) => Some(v as usize), }).unwrap_or(None), + field_id_read_enabled: false, // Default value }) } } diff --git a/datafusion/proto/src/logical_plan/file_formats.rs b/datafusion/proto/src/logical_plan/file_formats.rs index 08f42b0af7290..be314123985c9 100644 --- a/datafusion/proto/src/logical_plan/file_formats.rs +++ b/datafusion/proto/src/logical_plan/file_formats.rs @@ -525,6 +525,7 @@ mod parquet { max_predicate_cache_size: proto.max_predicate_cache_size_opt.as_ref().map(|opt| match opt { parquet_options::MaxPredicateCacheSizeOpt::MaxPredicateCacheSize(size) => *size as usize, }), + field_id_read_enabled: false, } } } From 69b86f7b68d629f46be10757c0aedac28341b1a0 Mon Sep 17 00:00:00 2001 From: Subham Singhal Date: Sun, 15 Feb 2026 23:41:01 +0530 Subject: [PATCH 04/15] Fix UT --- datafusion/datasource-parquet/src/source.rs | 8 ++++---- datafusion/sqllogictest/test_files/information_schema.slt | 2 ++ docs/source/user-guide/configs.md | 1 + 3 files changed, 7 insertions(+), 4 deletions(-) diff --git a/datafusion/datasource-parquet/src/source.rs b/datafusion/datasource-parquet/src/source.rs index b6e6a10bb27e3..0da157745dd7a 100644 --- a/datafusion/datasource-parquet/src/source.rs +++ b/datafusion/datasource-parquet/src/source.rs @@ -541,9 +541,6 @@ impl FileSource for ParquetSource { .as_ref() .map(|time_unit| parse_coerce_int96_string(time_unit.as_str()).unwrap()); - let field_id_read_enabled = - self.table_parquet_options.global.field_id_read_enabled; - let opener = Arc::new(ParquetOpener { partition_index: partition, projection: self.projection.clone(), @@ -571,7 +568,10 @@ impl FileSource for ParquetSource { encryption_factory: self.get_encryption_factory_with_config(), max_predicate_cache_size: self.max_predicate_cache_size(), reverse_row_groups: self.reverse_row_groups, - field_id_read_enabled, + field_id_read_enabled: self + .table_parquet_options + .global + .field_id_read_enabled, }); Ok(opener) } diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index b61ceecb24fc0..93678314904b7 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -244,6 +244,7 @@ datafusion.execution.parquet.dictionary_enabled true datafusion.execution.parquet.dictionary_page_size_limit 1048576 datafusion.execution.parquet.enable_page_index true datafusion.execution.parquet.encoding NULL +datafusion.execution.parquet.field_id_read_enabled false datafusion.execution.parquet.force_filter_selections false datafusion.execution.parquet.max_predicate_cache_size NULL datafusion.execution.parquet.max_row_group_size 1048576 @@ -382,6 +383,7 @@ datafusion.execution.parquet.dictionary_enabled true (writing) Sets if dictionar datafusion.execution.parquet.dictionary_page_size_limit 1048576 (writing) Sets best effort maximum dictionary page size, in bytes datafusion.execution.parquet.enable_page_index true (reading) If true, reads the Parquet data page level metadata (the Page Index), if present, to reduce the I/O and number of rows decoded. datafusion.execution.parquet.encoding NULL (writing) Sets default encoding for any column. Valid values are: plain, plain_dictionary, rle, bit_packed, delta_binary_packed, delta_length_byte_array, delta_byte_array, rle_dictionary, and byte_stream_split. These values are not case sensitive. If NULL, uses default parquet writer setting +datafusion.execution.parquet.field_id_read_enabled false (reading) If true, use Parquet field IDs for column resolution instead of column names. This enables schema evolution with renamed/reordered columns. When field IDs are unavailable, falls back to name-based matching. datafusion.execution.parquet.force_filter_selections false (reading) Force the use of RowSelections for filter results, when pushdown_filters is enabled. If false, the reader will automatically choose between a RowSelection and a Bitmap based on the number and pattern of selected rows. datafusion.execution.parquet.max_predicate_cache_size NULL (reading) The maximum predicate cache size, in bytes. When `pushdown_filters` is enabled, sets the maximum memory used to cache the results of predicate evaluation between filter evaluation and output generation. Decreasing this value will reduce memory usage, but may increase IO and CPU usage. None means use the default parquet reader setting. 0 means no caching. datafusion.execution.parquet.max_row_group_size 1048576 (writing) Target maximum number of rows in each row group (defaults to 1M rows). Writing larger row groups requires more memory to write, but can get better compression and be faster to read. diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index e48f0a7c92276..ca0f4afbffc97 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -92,6 +92,7 @@ The following configuration settings are available: | datafusion.execution.parquet.coerce_int96 | NULL | (reading) If true, parquet reader will read columns of physical type int96 as originating from a different resolution than nanosecond. This is useful for reading data from systems like Spark which stores microsecond resolution timestamps in an int96 allowing it to write values with a larger date range than 64-bit timestamps with nanosecond resolution. | | datafusion.execution.parquet.bloom_filter_on_read | true | (reading) Use any available bloom filters when reading parquet files | | datafusion.execution.parquet.max_predicate_cache_size | NULL | (reading) The maximum predicate cache size, in bytes. When `pushdown_filters` is enabled, sets the maximum memory used to cache the results of predicate evaluation between filter evaluation and output generation. Decreasing this value will reduce memory usage, but may increase IO and CPU usage. None means use the default parquet reader setting. 0 means no caching. | +| datafusion.execution.parquet.field_id_read_enabled | false | (reading) If true, use Parquet field IDs for column resolution instead of column names. This enables schema evolution with renamed/reordered columns. When field IDs are unavailable, falls back to name-based matching. | | datafusion.execution.parquet.data_pagesize_limit | 1048576 | (writing) Sets best effort maximum size of data page in bytes | | datafusion.execution.parquet.write_batch_size | 1024 | (writing) Sets write_batch_size in rows | | datafusion.execution.parquet.writer_version | 1.0 | (writing) Sets parquet writer version valid values are "1.0" and "2.0" | From d4e456e63666e9c93c5db38c4cb4919a9dbf2a5c Mon Sep 17 00:00:00 2001 From: Subham Singhal Date: Sun, 22 Feb 2026 11:32:28 +0530 Subject: [PATCH 05/15] Adds validation for complex types --- datafusion/common/src/config.rs | 30 +- datafusion/common/src/parquet_config.rs | 26 ++ datafusion/core/tests/parquet/field_id.rs | 322 +++++++++++++++++- .../datasource-parquet/src/file_format.rs | 11 +- datafusion/datasource-parquet/src/metadata.rs | 54 ++- datafusion/physical-expr/src/utils/mod.rs | 39 ++- 6 files changed, 422 insertions(+), 60 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 86956dfe523fd..e2873707e001a 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -3076,21 +3076,21 @@ config_namespace! { /// If not specified, the default level for the compression algorithm is used. pub compression_level: Option, default = None pub schema_infer_max_rec: Option, default = None - /// The JSON format to use when reading files. - /// - /// When `true` (default), expects newline-delimited JSON (NDJSON): - /// ```text - /// {"key1": 1, "key2": "val"} - /// {"key1": 2, "key2": "vals"} - /// ``` - /// - /// When `false`, expects JSON array format: - /// ```text - /// [ - /// {"key1": 1, "key2": "val"}, - /// {"key1": 2, "key2": "vals"} - /// ] - /// ``` + /// The JSON format to use when reading files. + /// + /// When `true` (default), expects newline-delimited JSON (NDJSON): + /// ```text + /// {"key1": 1, "key2": "val"} + /// {"key1": 2, "key2": "vals"} + /// ``` + /// + /// When `false`, expects JSON array format: + /// ```text + /// [ + /// {"key1": 1, "key2": "val"}, + /// {"key1": 2, "key2": "vals"} + /// ] + /// ``` pub newline_delimited: bool, default = true } } diff --git a/datafusion/common/src/parquet_config.rs b/datafusion/common/src/parquet_config.rs index 9d6d7a88566a7..f439e3a6d97f0 100644 --- a/datafusion/common/src/parquet_config.rs +++ b/datafusion/common/src/parquet_config.rs @@ -21,6 +21,32 @@ use std::str::FromStr; use crate::config::{ConfigField, Visit}; use crate::error::{DataFusionError, Result}; +/// Metadata key for storing Parquet field IDs in Arrow field metadata. +/// +/// Field IDs are stable identifiers for columns in Parquet files that enable +/// schema evolution with renamed or reordered columns. When `field_id_read_enabled` +/// is true, DataFusion stores field IDs from Parquet files in Arrow field metadata +/// using this key, allowing columns to be matched by ID instead of name. +/// +/// # Example +/// ```rust +/// use datafusion_common::parquet_config::PARQUET_FIELD_ID_META_KEY; +/// use arrow::datatypes::Field; +/// use std::collections::HashMap; +/// +/// let mut metadata = HashMap::new(); +/// metadata.insert(PARQUET_FIELD_ID_META_KEY.to_string(), "42".to_string()); +/// let field = Field::new("my_column", arrow::datatypes::DataType::Int32, false) +/// .with_metadata(metadata); +/// +/// // Later, retrieve the field ID +/// let field_id = field.metadata() +/// .get(PARQUET_FIELD_ID_META_KEY) +/// .and_then(|s| s.parse::().ok()); +/// assert_eq!(field_id, Some(42)); +/// ``` +pub const PARQUET_FIELD_ID_META_KEY: &str = "PARQUET:field_id"; + /// Parquet writer version options for controlling the Parquet file format version /// /// This enum validates parquet writer version values at configuration time, diff --git a/datafusion/core/tests/parquet/field_id.rs b/datafusion/core/tests/parquet/field_id.rs index f85a1447f512b..c594fbb3ec259 100644 --- a/datafusion/core/tests/parquet/field_id.rs +++ b/datafusion/core/tests/parquet/field_id.rs @@ -18,9 +18,9 @@ //! Integration tests for Parquet field ID support use arrow::array::{ - Array, Int32Array, Int64Array, RecordBatch, StringArray, StringViewArray, + Array, Int32Array, Int64Array, RecordBatch, StringArray, StringViewArray, StructArray, }; -use arrow::datatypes::{DataType, Field, Schema}; +use arrow::datatypes::{DataType, Field, Fields, Schema}; use datafusion::prelude::*; use datafusion_common::Result; use parquet::arrow::ArrowWriter; @@ -50,11 +50,13 @@ fn create_parquet_file_with_field_ids( /// Helper to create a schema with field IDs in metadata fn schema_with_field_ids(fields: Vec<(String, DataType, i32)>) -> Schema { + use datafusion_common::parquet_config::PARQUET_FIELD_ID_META_KEY; + let fields_with_ids: Vec = fields .into_iter() .map(|(name, dtype, field_id)| { let mut metadata = HashMap::new(); - metadata.insert("PARQUET:field_id".to_string(), field_id.to_string()); + metadata.insert(PARQUET_FIELD_ID_META_KEY.to_string(), field_id.to_string()); Field::new(name, dtype, false).with_metadata(metadata) }) .collect(); @@ -484,25 +486,29 @@ async fn test_aggregation_with_field_ids() -> Result<()> { } #[tokio::test] -async fn test_fallback_to_name_when_no_field_ids() -> Result<()> { +async fn test_schema_evolution_added_column() -> Result<()> { let tmp_dir = TempDir::new()?; let file_path = tmp_dir.path().join("test.parquet"); - // Create schema WITHOUT field IDs in metadata - let schema = Arc::new(Schema::new(vec![ - Field::new("user_id", DataType::Int64, false), - Field::new("amount", DataType::Int64, false), + // Write file with only 2 columns + let write_schema = Arc::new(schema_with_field_ids(vec![ + ("id".to_string(), DataType::Int32, 1), + ("name".to_string(), DataType::Utf8, 2), ])); let batch = RecordBatch::try_new( - Arc::clone(&schema), + Arc::clone(&write_schema), vec![ - Arc::new(Int64Array::from(vec![1, 2, 3])), - Arc::new(Int64Array::from(vec![100, 200, 300])), + Arc::new(Int32Array::from(vec![1, 2, 3])), + Arc::new(StringArray::from(vec!["Alice", "Bob", "Charlie"])), ], )?; - create_parquet_file_with_field_ids(file_path.to_str().unwrap(), schema, vec![batch])?; + create_parquet_file_with_field_ids( + file_path.to_str().unwrap(), + write_schema, + vec![batch], + )?; // Create context with field ID reading enabled let ctx = SessionContext::new(); @@ -518,19 +524,303 @@ async fn test_fallback_to_name_when_no_field_ids() -> Result<()> { ) .await?; - // Should fall back to name-based matching - let df = ctx.sql("SELECT user_id, amount FROM test").await?; + // Query should work - reading only the columns that exist + let df = ctx.sql("SELECT id, name FROM test").await?; let results = df.collect().await?; assert_eq!(results.len(), 1); assert_eq!(results[0].num_rows(), 3); + assert_eq!(results[0].num_columns(), 2); - let user_ids = results[0] + let id_vals = results[0] + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(id_vals.value(0), 1); + assert_eq!(id_vals.value(1), 2); + assert_eq!(id_vals.value(2), 3); + + // Get name column - might be StringArray or StringViewArray + let name_col = results[0].column(1); + let names: Vec<&str> = match name_col.data_type() { + DataType::Utf8 => name_col + .as_any() + .downcast_ref::() + .unwrap() + .iter() + .map(|v| v.unwrap()) + .collect(), + DataType::Utf8View => name_col + .as_any() + .downcast_ref::() + .unwrap() + .iter() + .map(|v| v.unwrap()) + .collect(), + _ => panic!("Unexpected data type for name column"), + }; + + assert_eq!(names, vec!["Alice", "Bob", "Charlie"]); + + Ok(()) +} + +#[tokio::test] +async fn test_schema_evolution_missing_column() -> Result<()> { + let tmp_dir = TempDir::new()?; + let file_path = tmp_dir.path().join("test.parquet"); + + // Write file with 3 columns + let write_schema = Arc::new(schema_with_field_ids(vec![ + ("id".to_string(), DataType::Int32, 1), + ("name".to_string(), DataType::Utf8, 2), + ("status".to_string(), DataType::Utf8, 3), + ])); + + let batch = RecordBatch::try_new( + Arc::clone(&write_schema), + vec![ + Arc::new(Int32Array::from(vec![1, 2, 3])), + Arc::new(StringArray::from(vec!["Alice", "Bob", "Charlie"])), + Arc::new(StringArray::from(vec!["active", "inactive", "active"])), + ], + )?; + + create_parquet_file_with_field_ids( + file_path.to_str().unwrap(), + write_schema, + vec![batch], + )?; + + // Create context with field ID reading enabled + let ctx = SessionContext::new(); + ctx.sql("SET datafusion.execution.parquet.field_id_read_enabled = true") + .await? + .collect() + .await?; + + ctx.register_parquet( + "test", + file_path.to_str().unwrap(), + ParquetReadOptions::default(), + ) + .await?; + + // Query should work - only reading columns id and name (skipping status) + let df = ctx.sql("SELECT id, name FROM test").await?; + let results = df.collect().await?; + + assert_eq!(results.len(), 1); + assert_eq!(results[0].num_rows(), 3); + assert_eq!(results[0].num_columns(), 2); + + let id_vals = results[0] .column(0) .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(id_vals.value(0), 1); + assert_eq!(id_vals.value(1), 2); + assert_eq!(id_vals.value(2), 3); + + // Verify we can still read all columns if we want + let df_all = ctx.sql("SELECT id, name, status FROM test").await?; + let results_all = df_all.collect().await?; + + assert_eq!(results_all.len(), 1); + assert_eq!(results_all[0].num_columns(), 3); + + Ok(()) +} + +#[tokio::test] +async fn test_schema_evolution_column_type_changed() -> Result<()> { + let tmp_dir = TempDir::new()?; + let old_file = tmp_dir.path().join("old.parquet"); + let new_file = tmp_dir.path().join("new.parquet"); + + // Write old file with Int32 for amount column (field_id=2) + let old_schema = Arc::new(schema_with_field_ids(vec![ + ("id".to_string(), DataType::Int32, 1), + ("amount".to_string(), DataType::Int32, 2), + ])); + + let old_batch = RecordBatch::try_new( + Arc::clone(&old_schema), + vec![ + Arc::new(Int32Array::from(vec![1, 2])), + Arc::new(Int32Array::from(vec![100, 200])), + ], + )?; + + create_parquet_file_with_field_ids( + old_file.to_str().unwrap(), + old_schema, + vec![old_batch], + )?; + + // Write new file with Int64 for amount column (field_id=2 - SAME field ID, different type) + let new_schema = Arc::new(schema_with_field_ids(vec![ + ("id".to_string(), DataType::Int32, 1), + ("amount".to_string(), DataType::Int64, 2), + ])); + + let new_batch = RecordBatch::try_new( + Arc::clone(&new_schema), + vec![ + Arc::new(Int32Array::from(vec![3, 4])), + Arc::new(Int64Array::from(vec![300, 400])), + ], + )?; + + create_parquet_file_with_field_ids( + new_file.to_str().unwrap(), + new_schema, + vec![new_batch], + )?; + + // Create context with field ID reading enabled + let ctx = SessionContext::new(); + ctx.sql("SET datafusion.execution.parquet.field_id_read_enabled = true") + .await? + .collect() + .await?; + + // Register both files with different table names + ctx.register_parquet( + "old_table", + old_file.to_str().unwrap(), + ParquetReadOptions::default(), + ) + .await?; + ctx.register_parquet( + "new_table", + new_file.to_str().unwrap(), + ParquetReadOptions::default(), + ) + .await?; + + // Query old file - should have Int32 amounts + let df_old = ctx.sql("SELECT id, amount FROM old_table").await?; + let results_old = df_old.collect().await?; + + assert_eq!(results_old.len(), 1); + let old_amounts = results_old[0] + .column(1) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(old_amounts.value(0), 100); + assert_eq!(old_amounts.value(1), 200); + + // Query new file - should have Int64 amounts + let df_new = ctx.sql("SELECT id, amount FROM new_table").await?; + let results_new = df_new.collect().await?; + + assert_eq!(results_new.len(), 1); + let new_amounts = results_new[0] + .column(1) + .as_any() .downcast_ref::() .unwrap(); - assert_eq!(user_ids.value(0), 1); + assert_eq!(new_amounts.value(0), 300); + assert_eq!(new_amounts.value(1), 400); + + Ok(()) +} + +/// Test to verify that nested struct types are rejected when field_id_read_enabled = true +#[tokio::test] +async fn test_nested_struct_with_field_ids_fails_with_clear_error() -> Result<()> { + let tmp_dir = TempDir::new()?; + let file_path = tmp_dir.path().join("nested.parquet"); + + // Create a schema with nested struct + let address_fields = Fields::from(vec![ + Field::new("street", DataType::Utf8, false), + Field::new("city", DataType::Utf8, false), + ]); + + let schema = Arc::new(Schema::new(vec![ + Field::new("address", DataType::Struct(address_fields.clone()), false), + Field::new("age", DataType::Int32, false), + ])); + + // Create test data + let street_array = Arc::new(StringArray::from(vec!["Main St", "Oak Ave"])); + let city_array = Arc::new(StringArray::from(vec!["NYC", "LA"])); + + let address_struct = + StructArray::new(address_fields, vec![street_array, city_array], None); + + let age_array = Arc::new(Int32Array::from(vec![30, 25])); + + let batch = RecordBatch::try_new( + Arc::clone(&schema), + vec![Arc::new(address_struct), age_array], + )?; + + // Write to Parquet + create_parquet_file_with_field_ids(file_path.to_str().unwrap(), schema, vec![batch])?; + + // Reading with field_id_read_enabled = false should work fine + let ctx = SessionContext::new(); + ctx.register_parquet( + "test", + file_path.to_str().unwrap(), + ParquetReadOptions::default(), + ) + .await?; + + let df = ctx.sql("SELECT age FROM test").await?; + let results = df.collect().await?; + assert_eq!(results.len(), 1); + assert_eq!(results[0].num_rows(), 2); + + // Clean up for next test + ctx.deregister_table("test")?; + + // Now enable field_id_read_enabled + ctx.sql("SET datafusion.execution.parquet.field_id_read_enabled = true") + .await? + .collect() + .await?; + + // This should now fail with a clear error message about nested types + let result = ctx + .register_parquet( + "test", + file_path.to_str().unwrap(), + ParquetReadOptions::default(), + ) + .await; + + match result { + Err(e) => { + let error_msg = e.to_string(); + assert!( + error_msg.contains("not yet supported for nested/complex types"), + "Expected error about nested types, got: {error_msg}", + ); + assert!( + error_msg.contains("address"), + "Error should mention the problematic field 'address'" + ); + assert!( + error_msg.contains("Struct"), + "Error should mention the Struct type" + ); + } + Ok(_) => { + // Try to query to see if it fails there + ctx.sql("SELECT age FROM test").await?; + + panic!( + "Expected error when registering Parquet file with nested types and field_id_read_enabled=true" + ); + } + } Ok(()) } diff --git a/datafusion/datasource-parquet/src/file_format.rs b/datafusion/datasource-parquet/src/file_format.rs index 32b63b9603089..789858175e252 100644 --- a/datafusion/datasource-parquet/src/file_format.rs +++ b/datafusion/datasource-parquet/src/file_format.rs @@ -27,6 +27,7 @@ use std::{fmt, vec}; use arrow::array::RecordBatch; use arrow::datatypes::{Fields, Schema, SchemaRef, TimeUnit}; +use datafusion_common::parquet_config::PARQUET_FIELD_ID_META_KEY; use datafusion_datasource::TableSchema; use datafusion_datasource::file_compression_type::FileCompressionType; use datafusion_datasource::file_sink_config::{FileSink, FileSinkConfig}; @@ -281,6 +282,11 @@ impl ParquetFormat { self.options.global.coerce_int96 = time_unit; self } + + /// Get whether field ID reading is enabled from options + pub fn field_id_read_enabled(&self) -> bool { + self.options.global.field_id_read_enabled + } } /// Clears all metadata (Schema level and field level) on an iterator @@ -385,6 +391,7 @@ impl FileFormat for ParquetFormat { .with_decryption_properties(file_decryption_properties) .with_file_metadata_cache(Some(Arc::clone(&file_metadata_cache))) .with_coerce_int96(coerce_int96) + .with_enable_field_ids(self.field_id_read_enabled()) .fetch_schema_with_location() .await?; Ok::<_, DataFusionError>(result) @@ -668,7 +675,7 @@ pub fn apply_file_schema_type_coercions( .iter() .filter_map(|f| { f.metadata() - .get("PARQUET:field_id") + .get(PARQUET_FIELD_ID_META_KEY) .and_then(|id_str| id_str.parse::().ok()) .map(|id| (id, f)) }) @@ -694,7 +701,7 @@ pub fn apply_file_schema_type_coercions( // Try field ID matching first field .metadata() - .get("PARQUET:field_id") + .get(PARQUET_FIELD_ID_META_KEY) .and_then(|id_str| id_str.parse::().ok()) .and_then(|id| table_field_by_id.get(&id)) .map(|f| f.data_type()) diff --git a/datafusion/datasource-parquet/src/metadata.rs b/datafusion/datasource-parquet/src/metadata.rs index b79827c2f9f49..41a3c0372253e 100644 --- a/datafusion/datasource-parquet/src/metadata.rs +++ b/datafusion/datasource-parquet/src/metadata.rs @@ -29,7 +29,7 @@ use arrow::datatypes::{DataType, Schema, SchemaRef, TimeUnit}; use datafusion_common::encryption::FileDecryptionProperties; use datafusion_common::stats::Precision; use datafusion_common::{ - ColumnStatistics, DataFusionError, Result, ScalarValue, Statistics, + ColumnStatistics, DataFusionError, Result, ScalarValue, Statistics, not_impl_err, }; use datafusion_execution::cache::cache_manager::{ CachedFileMetadataEntry, FileMetadata, FileMetadataCache, @@ -73,20 +73,38 @@ pub struct DFParquetMetadata<'a> { } /// Extracts Parquet field IDs and stores them in Arrow field metadata -/// under the key "PARQUET:field_id" +/// under the key `\[PARQUET_FIELD_ID_META_KEY`\] /// /// # Limitations /// -/// TODO: Currently only supports flat schemas (top-level primitive fields). -/// Nested field IDs within structs, lists, and maps are not yet supported. -/// This requires recursive traversal of the Parquet schema tree to extract -/// field IDs at all nesting levels. See PARQUET_FIELD_ID_IMPLEMENTATION.md -/// for details on nested schema support. +/// Currently only supports flat schemas (top-level primitive fields). +/// Returns an error if the schema contains nested types (structs, lists, maps). +/// +/// # Errors +/// +/// Returns an error if `arrow_schema` contains any complex/nested types when field IDs +/// are enabled, as these are not yet supported. +/// Nested type support see () fn add_field_ids_to_arrow_schema( arrow_schema: &Schema, parquet_schema: &SchemaDescriptor, ) -> Result { use arrow::datatypes::Field; + use datafusion_common::parquet_config::PARQUET_FIELD_ID_META_KEY; + + // Validate that schema is flat (no nested types) + // This prevents incorrect field ID assignment for complex types + for (idx, field) in arrow_schema.fields().iter().enumerate() { + if is_nested_type(field.data_type()) { + return not_impl_err!( + "Field ID reading is not yet supported for nested/complex types. \ + Field '{}' at index {} has type {:?}.", + field.name(), + idx, + field.data_type() + ); + } + } let fields_with_ids: Vec> = arrow_schema .fields() @@ -94,8 +112,6 @@ fn add_field_ids_to_arrow_schema( .enumerate() .map(|(idx, field)| { // Get the corresponding Parquet column descriptor - // TODO: This only works for flat schemas - parquet_schema.column(idx) - // returns leaf columns only, missing nested struct fields let col_desc = parquet_schema.column(idx); // Extract field ID from the schema type @@ -105,7 +121,8 @@ fn add_field_ids_to_arrow_schema( if field_id > 0 { // Add field ID to field metadata let mut metadata = field.metadata().clone(); - metadata.insert("PARQUET:field_id".to_string(), field_id.to_string()); + metadata + .insert(PARQUET_FIELD_ID_META_KEY.to_string(), field_id.to_string()); Arc::new(field.as_ref().clone().with_metadata(metadata)) } else { Arc::clone(field) @@ -119,6 +136,23 @@ fn add_field_ids_to_arrow_schema( )) } +/// Helper function to check if a data type is nested/complex +fn is_nested_type(data_type: &DataType) -> bool { + matches!( + data_type, + DataType::List(_) + | DataType::LargeList(_) + | DataType::FixedSizeList(_, _) + | DataType::Struct(_) + | DataType::Union(_, _) + | DataType::Map(_, _) + | DataType::Dictionary(_, _) + | DataType::RunEndEncoded(_, _) + | DataType::ListView(_) + | DataType::LargeListView(_) + ) +} + impl<'a> DFParquetMetadata<'a> { pub fn new(store: &'a dyn ObjectStore, object_meta: &'a ObjectMeta) -> Self { Self { diff --git a/datafusion/physical-expr/src/utils/mod.rs b/datafusion/physical-expr/src/utils/mod.rs index 54155e7fbb2e4..9abca794495a6 100644 --- a/datafusion/physical-expr/src/utils/mod.rs +++ b/datafusion/physical-expr/src/utils/mod.rs @@ -33,6 +33,7 @@ use datafusion_common::tree_node::{ use datafusion_common::{HashMap, HashSet, Result}; use datafusion_expr::Operator; +use datafusion_common::parquet_config::PARQUET_FIELD_ID_META_KEY; use petgraph::graph::NodeIndex; use petgraph::stable_graph::StableGraph; @@ -269,7 +270,7 @@ pub fn reassign_expr_columns( fn get_field_id(field: &arrow::datatypes::Field) -> Option { field .metadata() - .get("PARQUET:field_id") + .get(PARQUET_FIELD_ID_META_KEY) .and_then(|s| s.parse::().ok()) } @@ -367,6 +368,7 @@ pub(crate) mod tests { ColumnarValue, ScalarFunctionArgs, ScalarUDFImpl, Signature, Volatility, }; + use datafusion_common::parquet_config::PARQUET_FIELD_ID_META_KEY; use petgraph::visit::Bfs; #[derive(Debug, PartialEq, Eq, Hash)] @@ -656,7 +658,7 @@ pub(crate) mod tests { #[test] fn test_get_field_id_present() { let mut metadata = HashMap::new(); - metadata.insert("PARQUET:field_id".to_string(), "42".to_string()); + metadata.insert(PARQUET_FIELD_ID_META_KEY.to_string(), "42".to_string()); let field = Field::new("test", DataType::Int64, false).with_metadata(metadata); assert_eq!(get_field_id(&field), Some(42)); @@ -671,7 +673,10 @@ pub(crate) mod tests { #[test] fn test_get_field_id_invalid() { let mut metadata = HashMap::new(); - metadata.insert("PARQUET:field_id".to_string(), "not_a_number".to_string()); + metadata.insert( + PARQUET_FIELD_ID_META_KEY.to_string(), + "not_a_number".to_string(), + ); let field = Field::new("test", DataType::Int64, false).with_metadata(metadata); assert_eq!(get_field_id(&field), None); @@ -681,9 +686,9 @@ pub(crate) mod tests { fn test_find_field_index_by_field_id() -> Result<()> { // Source schema: field IDs present let mut metadata1 = HashMap::new(); - metadata1.insert("PARQUET:field_id".to_string(), "1".to_string()); + metadata1.insert(PARQUET_FIELD_ID_META_KEY.to_string(), "1".to_string()); let mut metadata2 = HashMap::new(); - metadata2.insert("PARQUET:field_id".to_string(), "2".to_string()); + metadata2.insert(PARQUET_FIELD_ID_META_KEY.to_string(), "2".to_string()); let source_schema = Schema::new(vec![ Field::new("user_id", DataType::Int64, false) @@ -718,11 +723,11 @@ pub(crate) mod tests { fn test_find_field_index_by_field_id_reordered() -> Result<()> { // Source schema: columns in order [a, b, c] let mut meta_a = HashMap::new(); - meta_a.insert("PARQUET:field_id".to_string(), "1".to_string()); + meta_a.insert(PARQUET_FIELD_ID_META_KEY.to_string(), "1".to_string()); let mut meta_b = HashMap::new(); - meta_b.insert("PARQUET:field_id".to_string(), "2".to_string()); + meta_b.insert(PARQUET_FIELD_ID_META_KEY.to_string(), "2".to_string()); let mut meta_c = HashMap::new(); - meta_c.insert("PARQUET:field_id".to_string(), "3".to_string()); + meta_c.insert(PARQUET_FIELD_ID_META_KEY.to_string(), "3".to_string()); let source_schema = Schema::new(vec![ Field::new("a", DataType::Int64, false).with_metadata(meta_a.clone()), @@ -776,7 +781,7 @@ pub(crate) mod tests { fn test_find_field_index_mixed_field_ids() -> Result<()> { // Source schema: some fields have IDs, some don't let mut metadata1 = HashMap::new(); - metadata1.insert("PARQUET:field_id".to_string(), "1".to_string()); + metadata1.insert(PARQUET_FIELD_ID_META_KEY.to_string(), "1".to_string()); let source_schema = Schema::new(vec![ Field::new("a", DataType::Int64, false).with_metadata(metadata1.clone()), @@ -812,11 +817,11 @@ pub(crate) mod tests { fn test_reassign_expr_columns_with_field_ids_simple() -> Result<()> { // Source schema: full file schema let mut meta1 = HashMap::new(); - meta1.insert("PARQUET:field_id".to_string(), "1".to_string()); + meta1.insert(PARQUET_FIELD_ID_META_KEY.to_string(), "1".to_string()); let mut meta2 = HashMap::new(); - meta2.insert("PARQUET:field_id".to_string(), "2".to_string()); + meta2.insert(PARQUET_FIELD_ID_META_KEY.to_string(), "2".to_string()); let mut meta3 = HashMap::new(); - meta3.insert("PARQUET:field_id".to_string(), "3".to_string()); + meta3.insert(PARQUET_FIELD_ID_META_KEY.to_string(), "3".to_string()); let source_schema = Schema::new(vec![ Field::new("user_id", DataType::Int64, false).with_metadata(meta1.clone()), @@ -852,11 +857,11 @@ pub(crate) mod tests { fn test_reassign_expr_columns_with_field_ids_complex() -> Result<()> { // Source schema let mut meta1 = HashMap::new(); - meta1.insert("PARQUET:field_id".to_string(), "1".to_string()); + meta1.insert(PARQUET_FIELD_ID_META_KEY.to_string(), "1".to_string()); let mut meta2 = HashMap::new(); - meta2.insert("PARQUET:field_id".to_string(), "2".to_string()); + meta2.insert(PARQUET_FIELD_ID_META_KEY.to_string(), "2".to_string()); let mut meta3 = HashMap::new(); - meta3.insert("PARQUET:field_id".to_string(), "3".to_string()); + meta3.insert(PARQUET_FIELD_ID_META_KEY.to_string(), "3".to_string()); let source_schema = Schema::new(vec![ Field::new("a", DataType::Int32, false).with_metadata(meta1.clone()), @@ -914,9 +919,9 @@ pub(crate) mod tests { fn test_reassign_expr_columns_with_field_ids_renamed_columns() -> Result<()> { // Source schema (file schema with old names) let mut meta1 = HashMap::new(); - meta1.insert("PARQUET:field_id".to_string(), "1".to_string()); + meta1.insert(PARQUET_FIELD_ID_META_KEY.to_string(), "1".to_string()); let mut meta2 = HashMap::new(); - meta2.insert("PARQUET:field_id".to_string(), "2".to_string()); + meta2.insert(PARQUET_FIELD_ID_META_KEY.to_string(), "2".to_string()); let source_schema = Schema::new(vec![ Field::new("user_id", DataType::Int64, false).with_metadata(meta1.clone()), From 111a87e2b81014817a944cca7181770d11edc2d3 Mon Sep 17 00:00:00 2001 From: Subham Singhal Date: Sun, 22 Feb 2026 15:33:13 +0530 Subject: [PATCH 06/15] Update field_id config name --- datafusion/common/src/config.rs | 4 +- .../common/src/file_options/parquet_writer.rs | 6 +- datafusion/common/src/parquet_config.rs | 2 +- datafusion/core/tests/parquet/field_id.rs | 28 +- .../datasource-parquet/src/file_format.rs | 6 +- datafusion/datasource-parquet/src/opener.rs | 14 +- datafusion/datasource-parquet/src/source.rs | 5 +- datafusion/proto-common/src/from_proto/mod.rs | 2 +- .../proto/src/logical_plan/file_formats.rs | 2 +- .../test_files/information_schema.slt | 2 +- docs/source/user-guide/configs.md | 258 +++++++++--------- 11 files changed, 163 insertions(+), 166 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index e2873707e001a..ec520577d0929 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -751,10 +751,10 @@ config_namespace! { /// parquet reader setting. 0 means no caching. pub max_predicate_cache_size: Option, default = None - /// (reading) If true, use Parquet field IDs for column resolution instead of + /// If true, use Parquet field IDs for column resolution instead of /// column names. This enables schema evolution with renamed/reordered columns. /// When field IDs are unavailable, falls back to name-based matching. - pub field_id_read_enabled: bool, default = false + pub field_id_enabled: bool, default = false // The following options affect writing to parquet files // and map to parquet::file::properties::WriterProperties diff --git a/datafusion/common/src/file_options/parquet_writer.rs b/datafusion/common/src/file_options/parquet_writer.rs index 85a281caa3ac3..0bb2e2edc4b89 100644 --- a/datafusion/common/src/file_options/parquet_writer.rs +++ b/datafusion/common/src/file_options/parquet_writer.rs @@ -209,7 +209,7 @@ impl ParquetOptions { coerce_int96: _, // not used for writer props skip_arrow_metadata: _, max_predicate_cache_size: _, - field_id_read_enabled: _, // not used for writer props + field_id_enabled: _, // not used for writer props } = self; let mut builder = WriterProperties::builder() @@ -461,7 +461,7 @@ mod tests { skip_arrow_metadata: defaults.skip_arrow_metadata, coerce_int96: None, max_predicate_cache_size: defaults.max_predicate_cache_size, - field_id_read_enabled: defaults.field_id_read_enabled, + field_id_enabled: defaults.field_id_enabled, } } @@ -576,7 +576,7 @@ mod tests { binary_as_string: global_options_defaults.binary_as_string, skip_arrow_metadata: global_options_defaults.skip_arrow_metadata, coerce_int96: None, - field_id_read_enabled: global_options_defaults.field_id_read_enabled, + field_id_enabled: global_options_defaults.field_id_enabled, }, column_specific_options, key_value_metadata, diff --git a/datafusion/common/src/parquet_config.rs b/datafusion/common/src/parquet_config.rs index f439e3a6d97f0..5356e1f930b48 100644 --- a/datafusion/common/src/parquet_config.rs +++ b/datafusion/common/src/parquet_config.rs @@ -24,7 +24,7 @@ use crate::error::{DataFusionError, Result}; /// Metadata key for storing Parquet field IDs in Arrow field metadata. /// /// Field IDs are stable identifiers for columns in Parquet files that enable -/// schema evolution with renamed or reordered columns. When `field_id_read_enabled` +/// schema evolution with renamed or reordered columns. When `field_id_enabled` /// is true, DataFusion stores field IDs from Parquet files in Arrow field metadata /// using this key, allowing columns to be matched by ID instead of name. /// diff --git a/datafusion/core/tests/parquet/field_id.rs b/datafusion/core/tests/parquet/field_id.rs index c594fbb3ec259..a6faa175204ce 100644 --- a/datafusion/core/tests/parquet/field_id.rs +++ b/datafusion/core/tests/parquet/field_id.rs @@ -90,7 +90,7 @@ async fn test_read_parquet_with_field_ids_enabled() -> Result<()> { // Create context with field ID reading enabled let ctx = SessionContext::new(); - ctx.sql("SET datafusion.execution.parquet.field_id_read_enabled = true") + ctx.sql("SET datafusion.execution.parquet.field_id_enabled = true") .await? .collect() .await?; @@ -190,7 +190,7 @@ async fn test_schema_evolution_renamed_columns() -> Result<()> { // Create context with field ID reading enabled let ctx = SessionContext::new(); - ctx.sql("SET datafusion.execution.parquet.field_id_read_enabled = true") + ctx.sql("SET datafusion.execution.parquet.field_id_enabled = true") .await? .collect() .await?; @@ -247,7 +247,7 @@ async fn test_schema_evolution_reordered_columns() -> Result<()> { // Create context with field ID reading enabled let ctx = SessionContext::new(); - ctx.sql("SET datafusion.execution.parquet.field_id_read_enabled = true") + ctx.sql("SET datafusion.execution.parquet.field_id_enabled = true") .await? .collect() .await?; @@ -315,7 +315,7 @@ async fn test_projection_with_field_ids() -> Result<()> { create_parquet_file_with_field_ids(file_path.to_str().unwrap(), schema, vec![batch])?; let ctx = SessionContext::new(); - ctx.sql("SET datafusion.execution.parquet.field_id_read_enabled = true") + ctx.sql("SET datafusion.execution.parquet.field_id_enabled = true") .await? .collect() .await?; @@ -372,7 +372,7 @@ async fn test_filter_with_field_ids() -> Result<()> { create_parquet_file_with_field_ids(file_path.to_str().unwrap(), schema, vec![batch])?; let ctx = SessionContext::new(); - ctx.sql("SET datafusion.execution.parquet.field_id_read_enabled = true") + ctx.sql("SET datafusion.execution.parquet.field_id_enabled = true") .await? .collect() .await?; @@ -426,7 +426,7 @@ async fn test_aggregation_with_field_ids() -> Result<()> { create_parquet_file_with_field_ids(file_path.to_str().unwrap(), schema, vec![batch])?; let ctx = SessionContext::new(); - ctx.sql("SET datafusion.execution.parquet.field_id_read_enabled = true") + ctx.sql("SET datafusion.execution.parquet.field_id_enabled = true") .await? .collect() .await?; @@ -512,7 +512,7 @@ async fn test_schema_evolution_added_column() -> Result<()> { // Create context with field ID reading enabled let ctx = SessionContext::new(); - ctx.sql("SET datafusion.execution.parquet.field_id_read_enabled = true") + ctx.sql("SET datafusion.execution.parquet.field_id_enabled = true") .await? .collect() .await?; @@ -595,7 +595,7 @@ async fn test_schema_evolution_missing_column() -> Result<()> { // Create context with field ID reading enabled let ctx = SessionContext::new(); - ctx.sql("SET datafusion.execution.parquet.field_id_read_enabled = true") + ctx.sql("SET datafusion.execution.parquet.field_id_enabled = true") .await? .collect() .await?; @@ -682,7 +682,7 @@ async fn test_schema_evolution_column_type_changed() -> Result<()> { // Create context with field ID reading enabled let ctx = SessionContext::new(); - ctx.sql("SET datafusion.execution.parquet.field_id_read_enabled = true") + ctx.sql("SET datafusion.execution.parquet.field_id_enabled = true") .await? .collect() .await?; @@ -730,7 +730,7 @@ async fn test_schema_evolution_column_type_changed() -> Result<()> { Ok(()) } -/// Test to verify that nested struct types are rejected when field_id_read_enabled = true +/// Test to verify that nested struct types are rejected when field_id_enabled = true #[tokio::test] async fn test_nested_struct_with_field_ids_fails_with_clear_error() -> Result<()> { let tmp_dir = TempDir::new()?; @@ -764,7 +764,7 @@ async fn test_nested_struct_with_field_ids_fails_with_clear_error() -> Result<() // Write to Parquet create_parquet_file_with_field_ids(file_path.to_str().unwrap(), schema, vec![batch])?; - // Reading with field_id_read_enabled = false should work fine + // Reading with field_id_enabled = false should work fine let ctx = SessionContext::new(); ctx.register_parquet( "test", @@ -781,8 +781,8 @@ async fn test_nested_struct_with_field_ids_fails_with_clear_error() -> Result<() // Clean up for next test ctx.deregister_table("test")?; - // Now enable field_id_read_enabled - ctx.sql("SET datafusion.execution.parquet.field_id_read_enabled = true") + // Now enable field_id_enabled + ctx.sql("SET datafusion.execution.parquet.field_id_enabled = true") .await? .collect() .await?; @@ -817,7 +817,7 @@ async fn test_nested_struct_with_field_ids_fails_with_clear_error() -> Result<() ctx.sql("SELECT age FROM test").await?; panic!( - "Expected error when registering Parquet file with nested types and field_id_read_enabled=true" + "Expected error when registering Parquet file with nested types and field_id_enabled=true" ); } } diff --git a/datafusion/datasource-parquet/src/file_format.rs b/datafusion/datasource-parquet/src/file_format.rs index 789858175e252..e0fbe356db3b4 100644 --- a/datafusion/datasource-parquet/src/file_format.rs +++ b/datafusion/datasource-parquet/src/file_format.rs @@ -284,8 +284,8 @@ impl ParquetFormat { } /// Get whether field ID reading is enabled from options - pub fn field_id_read_enabled(&self) -> bool { - self.options.global.field_id_read_enabled + pub fn field_id_enabled(&self) -> bool { + self.options.global.field_id_enabled } } @@ -391,7 +391,7 @@ impl FileFormat for ParquetFormat { .with_decryption_properties(file_decryption_properties) .with_file_metadata_cache(Some(Arc::clone(&file_metadata_cache))) .with_coerce_int96(coerce_int96) - .with_enable_field_ids(self.field_id_read_enabled()) + .with_enable_field_ids(self.field_id_enabled()) .fetch_schema_with_location() .await?; Ok::<_, DataFusionError>(result) diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index 16bbde344ca95..87d0a5b67b561 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -123,7 +123,7 @@ pub(super) struct ParquetOpener { /// Whether to read row groups in reverse order pub reverse_row_groups: bool, /// Whether to use Parquet field IDs for column resolution - pub field_id_read_enabled: bool, + pub field_id_enabled: bool, } /// Represents a prepared access plan with optional row selection @@ -209,7 +209,7 @@ impl FileOpener for ParquetOpener { )?; let batch_size = self.batch_size; - let field_id_read_enabled = self.field_id_read_enabled; + let field_id_enabled = self.field_id_enabled; // Calculate the output schema from the original projection (before literal replacement) // so we get correct field names from column references @@ -383,7 +383,7 @@ impl FileOpener for ParquetOpener { if let Some(merged) = apply_file_schema_type_coercions( &logical_file_schema, &physical_file_schema, - field_id_read_enabled, + field_id_enabled, ) { physical_file_schema = Arc::new(merged); options = options.with_schema(Arc::clone(&physical_file_schema)); @@ -630,7 +630,7 @@ impl FileOpener for ParquetOpener { // Rebase column indices to match the narrowed stream schema. // The projection expressions have indices based on physical_file_schema, // but the stream only contains the columns selected by the ProjectionMask. - let projection = if field_id_read_enabled { + let projection = if field_id_enabled { projection.try_map_exprs(|expr| { reassign_expr_columns_with_field_ids( expr, @@ -1080,7 +1080,7 @@ mod test { max_predicate_cache_size: Option, reverse_row_groups: bool, preserve_order: bool, - field_id_read_enabled: bool, + field_id_enabled: bool, } impl ParquetOpenerBuilder { @@ -1107,7 +1107,7 @@ mod test { max_predicate_cache_size: None, reverse_row_groups: false, preserve_order: false, - field_id_read_enabled: false, + field_id_enabled: false, } } @@ -1215,7 +1215,7 @@ mod test { encryption_factory: None, max_predicate_cache_size: self.max_predicate_cache_size, reverse_row_groups: self.reverse_row_groups, - field_id_read_enabled: self.field_id_read_enabled, + field_id_enabled: self.field_id_enabled, preserve_order: self.preserve_order, } } diff --git a/datafusion/datasource-parquet/src/source.rs b/datafusion/datasource-parquet/src/source.rs index 0da157745dd7a..bd68f7a9163c9 100644 --- a/datafusion/datasource-parquet/src/source.rs +++ b/datafusion/datasource-parquet/src/source.rs @@ -568,10 +568,7 @@ impl FileSource for ParquetSource { encryption_factory: self.get_encryption_factory_with_config(), max_predicate_cache_size: self.max_predicate_cache_size(), reverse_row_groups: self.reverse_row_groups, - field_id_read_enabled: self - .table_parquet_options - .global - .field_id_read_enabled, + field_id_enabled: self.table_parquet_options.global.field_id_enabled, }); Ok(opener) } diff --git a/datafusion/proto-common/src/from_proto/mod.rs b/datafusion/proto-common/src/from_proto/mod.rs index 294b6c1231e46..487d6c40565da 100644 --- a/datafusion/proto-common/src/from_proto/mod.rs +++ b/datafusion/proto-common/src/from_proto/mod.rs @@ -1090,7 +1090,7 @@ impl TryFrom<&protobuf::ParquetOptions> for ParquetOptions { max_predicate_cache_size: value.max_predicate_cache_size_opt.map(|opt| match opt { protobuf::parquet_options::MaxPredicateCacheSizeOpt::MaxPredicateCacheSize(v) => Some(v as usize), }).unwrap_or(None), - field_id_read_enabled: false, // Default value + field_id_enabled: false, // Default value }) } } diff --git a/datafusion/proto/src/logical_plan/file_formats.rs b/datafusion/proto/src/logical_plan/file_formats.rs index be314123985c9..cf7142bc7cfde 100644 --- a/datafusion/proto/src/logical_plan/file_formats.rs +++ b/datafusion/proto/src/logical_plan/file_formats.rs @@ -525,7 +525,7 @@ mod parquet { max_predicate_cache_size: proto.max_predicate_cache_size_opt.as_ref().map(|opt| match opt { parquet_options::MaxPredicateCacheSizeOpt::MaxPredicateCacheSize(size) => *size as usize, }), - field_id_read_enabled: false, + field_id_enabled: false, } } } diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index 93678314904b7..ba570ba192b0e 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -244,7 +244,7 @@ datafusion.execution.parquet.dictionary_enabled true datafusion.execution.parquet.dictionary_page_size_limit 1048576 datafusion.execution.parquet.enable_page_index true datafusion.execution.parquet.encoding NULL -datafusion.execution.parquet.field_id_read_enabled false +datafusion.execution.parquet.field_id_enabled false datafusion.execution.parquet.force_filter_selections false datafusion.execution.parquet.max_predicate_cache_size NULL datafusion.execution.parquet.max_row_group_size 1048576 diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index ca0f4afbffc97..e925f5f689f66 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -63,136 +63,136 @@ SET datafusion.execution.target_partitions = '1'; The following configuration settings are available: -| key | default | description | -| ----------------------------------------------------------------------- | ------------------------- | -------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | -| datafusion.catalog.create_default_catalog_and_schema | true | Whether the default catalog and schema should be created automatically. | -| datafusion.catalog.default_catalog | datafusion | The default catalog name - this impacts what SQL queries use if not specified | -| datafusion.catalog.default_schema | public | The default schema name - this impacts what SQL queries use if not specified | -| datafusion.catalog.information_schema | false | Should DataFusion provide access to `information_schema` virtual tables for displaying schema information | -| datafusion.catalog.location | NULL | Location scanned to load tables for `default` schema | -| datafusion.catalog.format | NULL | Type of `TableProvider` to use when loading `default` schema | -| datafusion.catalog.has_header | true | Default value for `format.has_header` for `CREATE EXTERNAL TABLE` if not specified explicitly in the statement. | -| datafusion.catalog.newlines_in_values | false | Specifies whether newlines in (quoted) CSV values are supported. This is the default value for `format.newlines_in_values` for `CREATE EXTERNAL TABLE` if not specified explicitly in the statement. Parsing newlines in quoted values may be affected by execution behaviour such as parallel file scanning. Setting this to `true` ensures that newlines in values are parsed successfully, which may reduce performance. | -| datafusion.execution.batch_size | 8192 | Default batch size while creating new batches, it's especially useful for buffer-in-memory batches since creating tiny batches would result in too much metadata memory consumption | -| datafusion.execution.perfect_hash_join_small_build_threshold | 1024 | A perfect hash join (see `HashJoinExec` for more details) will be considered if the range of keys (max - min) on the build side is < this threshold. This provides a fast path for joins with very small key ranges, bypassing the density check. Currently only supports cases where build_side.num_rows() < u32::MAX. Support for build_side.num_rows() >= u32::MAX will be added in the future. | -| datafusion.execution.perfect_hash_join_min_key_density | 0.15 | The minimum required density of join keys on the build side to consider a perfect hash join (see `HashJoinExec` for more details). Density is calculated as: `(number of rows) / (max_key - min_key + 1)`. A perfect hash join may be used if the actual key density > this value. Currently only supports cases where build_side.num_rows() < u32::MAX. Support for build_side.num_rows() >= u32::MAX will be added in the future. | -| datafusion.execution.coalesce_batches | true | When set to true, record batches will be examined between each operator and small batches will be coalesced into larger batches. This is helpful when there are highly selective filters or joins that could produce tiny output batches. The target batch size is determined by the configuration setting | -| datafusion.execution.collect_statistics | true | Should DataFusion collect statistics when first creating a table. Has no effect after the table is created. Applies to the default `ListingTableProvider` in DataFusion. Defaults to true. | -| datafusion.execution.target_partitions | 0 | Number of partitions for query execution. Increasing partitions can increase concurrency. Defaults to the number of CPU cores on the system | -| datafusion.execution.time_zone | NULL | The default time zone Some functions, e.g. `now` return timestamps in this time zone | -| datafusion.execution.parquet.enable_page_index | true | (reading) If true, reads the Parquet data page level metadata (the Page Index), if present, to reduce the I/O and number of rows decoded. | -| datafusion.execution.parquet.pruning | true | (reading) If true, the parquet reader attempts to skip entire row groups based on the predicate in the query and the metadata (min/max values) stored in the parquet file | -| datafusion.execution.parquet.skip_metadata | true | (reading) If true, the parquet reader skip the optional embedded metadata that may be in the file Schema. This setting can help avoid schema conflicts when querying multiple parquet files with schemas containing compatible types but different metadata | -| datafusion.execution.parquet.metadata_size_hint | 524288 | (reading) If specified, the parquet reader will try and fetch the last `size_hint` bytes of the parquet file optimistically. If not specified, two reads are required: One read to fetch the 8-byte parquet footer and another to fetch the metadata length encoded in the footer Default setting to 512 KiB, which should be sufficient for most parquet files, it can reduce one I/O operation per parquet file. If the metadata is larger than the hint, two reads will still be performed. | -| datafusion.execution.parquet.pushdown_filters | false | (reading) If true, filter expressions are be applied during the parquet decoding operation to reduce the number of rows decoded. This optimization is sometimes called "late materialization". | -| datafusion.execution.parquet.reorder_filters | false | (reading) If true, filter expressions evaluated during the parquet decoding operation will be reordered heuristically to minimize the cost of evaluation. If false, the filters are applied in the same order as written in the query | -| datafusion.execution.parquet.force_filter_selections | false | (reading) Force the use of RowSelections for filter results, when pushdown_filters is enabled. If false, the reader will automatically choose between a RowSelection and a Bitmap based on the number and pattern of selected rows. | -| datafusion.execution.parquet.schema_force_view_types | true | (reading) If true, parquet reader will read columns of `Utf8/Utf8Large` with `Utf8View`, and `Binary/BinaryLarge` with `BinaryView`. | -| datafusion.execution.parquet.binary_as_string | false | (reading) If true, parquet reader will read columns of `Binary/LargeBinary` with `Utf8`, and `BinaryView` with `Utf8View`. Parquet files generated by some legacy writers do not correctly set the UTF8 flag for strings, causing string columns to be loaded as BLOB instead. | -| datafusion.execution.parquet.coerce_int96 | NULL | (reading) If true, parquet reader will read columns of physical type int96 as originating from a different resolution than nanosecond. This is useful for reading data from systems like Spark which stores microsecond resolution timestamps in an int96 allowing it to write values with a larger date range than 64-bit timestamps with nanosecond resolution. | -| datafusion.execution.parquet.bloom_filter_on_read | true | (reading) Use any available bloom filters when reading parquet files | -| datafusion.execution.parquet.max_predicate_cache_size | NULL | (reading) The maximum predicate cache size, in bytes. When `pushdown_filters` is enabled, sets the maximum memory used to cache the results of predicate evaluation between filter evaluation and output generation. Decreasing this value will reduce memory usage, but may increase IO and CPU usage. None means use the default parquet reader setting. 0 means no caching. | -| datafusion.execution.parquet.field_id_read_enabled | false | (reading) If true, use Parquet field IDs for column resolution instead of column names. This enables schema evolution with renamed/reordered columns. When field IDs are unavailable, falls back to name-based matching. | -| datafusion.execution.parquet.data_pagesize_limit | 1048576 | (writing) Sets best effort maximum size of data page in bytes | -| datafusion.execution.parquet.write_batch_size | 1024 | (writing) Sets write_batch_size in rows | -| datafusion.execution.parquet.writer_version | 1.0 | (writing) Sets parquet writer version valid values are "1.0" and "2.0" | -| datafusion.execution.parquet.skip_arrow_metadata | false | (writing) Skip encoding the embedded arrow metadata in the KV_meta This is analogous to the `ArrowWriterOptions::with_skip_arrow_metadata`. Refer to | -| datafusion.execution.parquet.compression | zstd(3) | (writing) Sets default parquet compression codec. Valid values are: uncompressed, snappy, gzip(level), brotli(level), lz4, zstd(level), and lz4_raw. These values are not case sensitive. If NULL, uses default parquet writer setting Note that this default setting is not the same as the default parquet writer setting. | -| datafusion.execution.parquet.dictionary_enabled | true | (writing) Sets if dictionary encoding is enabled. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.dictionary_page_size_limit | 1048576 | (writing) Sets best effort maximum dictionary page size, in bytes | -| datafusion.execution.parquet.statistics_enabled | page | (writing) Sets if statistics are enabled for any column Valid values are: "none", "chunk", and "page" These values are not case sensitive. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.max_row_group_size | 1048576 | (writing) Target maximum number of rows in each row group (defaults to 1M rows). Writing larger row groups requires more memory to write, but can get better compression and be faster to read. | -| datafusion.execution.parquet.created_by | datafusion version 52.1.0 | (writing) Sets "created by" property | -| datafusion.execution.parquet.column_index_truncate_length | 64 | (writing) Sets column index truncate length | -| datafusion.execution.parquet.statistics_truncate_length | 64 | (writing) Sets statistics truncate length. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.data_page_row_count_limit | 20000 | (writing) Sets best effort maximum number of rows in data page | -| datafusion.execution.parquet.encoding | NULL | (writing) Sets default encoding for any column. Valid values are: plain, plain_dictionary, rle, bit_packed, delta_binary_packed, delta_length_byte_array, delta_byte_array, rle_dictionary, and byte_stream_split. These values are not case sensitive. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.bloom_filter_on_write | false | (writing) Write bloom filters for all columns when creating parquet files | -| datafusion.execution.parquet.bloom_filter_fpp | NULL | (writing) Sets bloom filter false positive probability. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.bloom_filter_ndv | NULL | (writing) Sets bloom filter number of distinct values. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.allow_single_file_parallelism | true | (writing) Controls whether DataFusion will attempt to speed up writing parquet files by serializing them in parallel. Each column in each row group in each output file are serialized in parallel leveraging a maximum possible core count of n_files*n_row_groups*n_columns. | -| datafusion.execution.parquet.maximum_parallel_row_group_writers | 1 | (writing) By default parallel parquet writer is tuned for minimum memory usage in a streaming execution plan. You may see a performance benefit when writing large parquet files by increasing maximum_parallel_row_group_writers and maximum_buffered_record_batches_per_stream if your system has idle cores and can tolerate additional memory usage. Boosting these values is likely worthwhile when writing out already in-memory data, such as from a cached data frame. | +| key | default | description | +| ------------------------------------------------------------------ | ------------------------- | -------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | +| datafusion.catalog.create_default_catalog_and_schema | true | Whether the default catalog and schema should be created automatically. | +| datafusion.catalog.default_catalog | datafusion | The default catalog name - this impacts what SQL queries use if not specified | +| datafusion.catalog.default_schema | public | The default schema name - this impacts what SQL queries use if not specified | +| datafusion.catalog.information_schema | false | Should DataFusion provide access to `information_schema` virtual tables for displaying schema information | +| datafusion.catalog.location | NULL | Location scanned to load tables for `default` schema | +| datafusion.catalog.format | NULL | Type of `TableProvider` to use when loading `default` schema | +| datafusion.catalog.has_header | true | Default value for `format.has_header` for `CREATE EXTERNAL TABLE` if not specified explicitly in the statement. | +| datafusion.catalog.newlines_in_values | false | Specifies whether newlines in (quoted) CSV values are supported. This is the default value for `format.newlines_in_values` for `CREATE EXTERNAL TABLE` if not specified explicitly in the statement. Parsing newlines in quoted values may be affected by execution behaviour such as parallel file scanning. Setting this to `true` ensures that newlines in values are parsed successfully, which may reduce performance. | +| datafusion.execution.batch_size | 8192 | Default batch size while creating new batches, it's especially useful for buffer-in-memory batches since creating tiny batches would result in too much metadata memory consumption | +| datafusion.execution.perfect_hash_join_small_build_threshold | 1024 | A perfect hash join (see `HashJoinExec` for more details) will be considered if the range of keys (max - min) on the build side is < this threshold. This provides a fast path for joins with very small key ranges, bypassing the density check. Currently only supports cases where build_side.num_rows() < u32::MAX. Support for build_side.num_rows() >= u32::MAX will be added in the future. | +| datafusion.execution.perfect_hash_join_min_key_density | 0.15 | The minimum required density of join keys on the build side to consider a perfect hash join (see `HashJoinExec` for more details). Density is calculated as: `(number of rows) / (max_key - min_key + 1)`. A perfect hash join may be used if the actual key density > this value. Currently only supports cases where build_side.num_rows() < u32::MAX. Support for build_side.num_rows() >= u32::MAX will be added in the future. | +| datafusion.execution.coalesce_batches | true | When set to true, record batches will be examined between each operator and small batches will be coalesced into larger batches. This is helpful when there are highly selective filters or joins that could produce tiny output batches. The target batch size is determined by the configuration setting | +| datafusion.execution.collect_statistics | true | Should DataFusion collect statistics when first creating a table. Has no effect after the table is created. Applies to the default `ListingTableProvider` in DataFusion. Defaults to true. | +| datafusion.execution.target_partitions | 0 | Number of partitions for query execution. Increasing partitions can increase concurrency. Defaults to the number of CPU cores on the system | +| datafusion.execution.time_zone | NULL | The default time zone Some functions, e.g. `now` return timestamps in this time zone | +| datafusion.execution.parquet.enable_page_index | true | (reading) If true, reads the Parquet data page level metadata (the Page Index), if present, to reduce the I/O and number of rows decoded. | +| datafusion.execution.parquet.pruning | true | (reading) If true, the parquet reader attempts to skip entire row groups based on the predicate in the query and the metadata (min/max values) stored in the parquet file | +| datafusion.execution.parquet.skip_metadata | true | (reading) If true, the parquet reader skip the optional embedded metadata that may be in the file Schema. This setting can help avoid schema conflicts when querying multiple parquet files with schemas containing compatible types but different metadata | +| datafusion.execution.parquet.metadata_size_hint | 524288 | (reading) If specified, the parquet reader will try and fetch the last `size_hint` bytes of the parquet file optimistically. If not specified, two reads are required: One read to fetch the 8-byte parquet footer and another to fetch the metadata length encoded in the footer Default setting to 512 KiB, which should be sufficient for most parquet files, it can reduce one I/O operation per parquet file. If the metadata is larger than the hint, two reads will still be performed. | +| datafusion.execution.parquet.pushdown_filters | false | (reading) If true, filter expressions are be applied during the parquet decoding operation to reduce the number of rows decoded. This optimization is sometimes called "late materialization". | +| datafusion.execution.parquet.reorder_filters | false | (reading) If true, filter expressions evaluated during the parquet decoding operation will be reordered heuristically to minimize the cost of evaluation. If false, the filters are applied in the same order as written in the query | +| datafusion.execution.parquet.force_filter_selections | false | (reading) Force the use of RowSelections for filter results, when pushdown_filters is enabled. If false, the reader will automatically choose between a RowSelection and a Bitmap based on the number and pattern of selected rows. | +| datafusion.execution.parquet.schema_force_view_types | true | (reading) If true, parquet reader will read columns of `Utf8/Utf8Large` with `Utf8View`, and `Binary/BinaryLarge` with `BinaryView`. | +| datafusion.execution.parquet.binary_as_string | false | (reading) If true, parquet reader will read columns of `Binary/LargeBinary` with `Utf8`, and `BinaryView` with `Utf8View`. Parquet files generated by some legacy writers do not correctly set the UTF8 flag for strings, causing string columns to be loaded as BLOB instead. | +| datafusion.execution.parquet.coerce_int96 | NULL | (reading) If true, parquet reader will read columns of physical type int96 as originating from a different resolution than nanosecond. This is useful for reading data from systems like Spark which stores microsecond resolution timestamps in an int96 allowing it to write values with a larger date range than 64-bit timestamps with nanosecond resolution. | +| datafusion.execution.parquet.bloom_filter_on_read | true | (reading) Use any available bloom filters when reading parquet files | +| datafusion.execution.parquet.max_predicate_cache_size | NULL | (reading) The maximum predicate cache size, in bytes. When `pushdown_filters` is enabled, sets the maximum memory used to cache the results of predicate evaluation between filter evaluation and output generation. Decreasing this value will reduce memory usage, but may increase IO and CPU usage. None means use the default parquet reader setting. 0 means no caching. | +| datafusion.execution.parquet.field_id_enabled | false | (reading) If true, use Parquet field IDs for column resolution instead of column names. This enables schema evolution with renamed/reordered columns. When field IDs are unavailable, falls back to name-based matching. | +| datafusion.execution.parquet.data_pagesize_limit | 1048576 | (writing) Sets best effort maximum size of data page in bytes | +| datafusion.execution.parquet.write_batch_size | 1024 | (writing) Sets write_batch_size in rows | +| datafusion.execution.parquet.writer_version | 1.0 | (writing) Sets parquet writer version valid values are "1.0" and "2.0" | +| datafusion.execution.parquet.skip_arrow_metadata | false | (writing) Skip encoding the embedded arrow metadata in the KV_meta This is analogous to the `ArrowWriterOptions::with_skip_arrow_metadata`. Refer to | +| datafusion.execution.parquet.compression | zstd(3) | (writing) Sets default parquet compression codec. Valid values are: uncompressed, snappy, gzip(level), brotli(level), lz4, zstd(level), and lz4_raw. These values are not case sensitive. If NULL, uses default parquet writer setting Note that this default setting is not the same as the default parquet writer setting. | +| datafusion.execution.parquet.dictionary_enabled | true | (writing) Sets if dictionary encoding is enabled. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.dictionary_page_size_limit | 1048576 | (writing) Sets best effort maximum dictionary page size, in bytes | +| datafusion.execution.parquet.statistics_enabled | page | (writing) Sets if statistics are enabled for any column Valid values are: "none", "chunk", and "page" These values are not case sensitive. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.max_row_group_size | 1048576 | (writing) Target maximum number of rows in each row group (defaults to 1M rows). Writing larger row groups requires more memory to write, but can get better compression and be faster to read. | +| datafusion.execution.parquet.created_by | datafusion version 52.1.0 | (writing) Sets "created by" property | +| datafusion.execution.parquet.column_index_truncate_length | 64 | (writing) Sets column index truncate length | +| datafusion.execution.parquet.statistics_truncate_length | 64 | (writing) Sets statistics truncate length. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.data_page_row_count_limit | 20000 | (writing) Sets best effort maximum number of rows in data page | +| datafusion.execution.parquet.encoding | NULL | (writing) Sets default encoding for any column. Valid values are: plain, plain_dictionary, rle, bit_packed, delta_binary_packed, delta_length_byte_array, delta_byte_array, rle_dictionary, and byte_stream_split. These values are not case sensitive. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.bloom_filter_on_write | false | (writing) Write bloom filters for all columns when creating parquet files | +| datafusion.execution.parquet.bloom_filter_fpp | NULL | (writing) Sets bloom filter false positive probability. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.bloom_filter_ndv | NULL | (writing) Sets bloom filter number of distinct values. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.allow_single_file_parallelism | true | (writing) Controls whether DataFusion will attempt to speed up writing parquet files by serializing them in parallel. Each column in each row group in each output file are serialized in parallel leveraging a maximum possible core count of n_files*n_row_groups*n_columns. | +| datafusion.execution.parquet.maximum_parallel_row_group_writers | 1 | (writing) By default parallel parquet writer is tuned for minimum memory usage in a streaming execution plan. You may see a performance benefit when writing large parquet files by increasing maximum_parallel_row_group_writers and maximum_buffered_record_batches_per_stream if your system has idle cores and can tolerate additional memory usage. Boosting these values is likely worthwhile when writing out already in-memory data, such as from a cached data frame. | | datafusion.execution.parquet.maximum_buffered_record_batches_per_stream | 2 | (writing) By default parallel parquet writer is tuned for minimum memory usage in a streaming execution plan. You may see a performance benefit when writing large parquet files by increasing maximum_parallel_row_group_writers and maximum_buffered_record_batches_per_stream if your system has idle cores and can tolerate additional memory usage. Boosting these values is likely worthwhile when writing out already in-memory data, such as from a cached data frame. | -| datafusion.execution.planning_concurrency | 0 | Fan-out during initial physical planning. This is mostly use to plan `UNION` children in parallel. Defaults to the number of CPU cores on the system | -| datafusion.execution.skip_physical_aggregate_schema_check | false | When set to true, skips verifying that the schema produced by planning the input of `LogicalPlan::Aggregate` exactly matches the schema of the input plan. When set to false, if the schema does not match exactly (including nullability and metadata), a planning error will be raised. This is used to workaround bugs in the planner that are now caught by the new schema verification step. | -| datafusion.execution.spill_compression | uncompressed | Sets the compression codec used when spilling data to disk. Since datafusion writes spill files using the Arrow IPC Stream format, only codecs supported by the Arrow IPC Stream Writer are allowed. Valid values are: uncompressed, lz4_frame, zstd. Note: lz4_frame offers faster (de)compression, but typically results in larger spill files. In contrast, zstd achieves higher compression ratios at the cost of slower (de)compression speed. | -| datafusion.execution.sort_spill_reservation_bytes | 10485760 | Specifies the reserved memory for each spillable sort operation to facilitate an in-memory merge. When a sort operation spills to disk, the in-memory data must be sorted and merged before being written to a file. This setting reserves a specific amount of memory for that in-memory sort/merge process. Note: This setting is irrelevant if the sort operation cannot spill (i.e., if there's no `DiskManager` configured). | -| datafusion.execution.sort_in_place_threshold_bytes | 1048576 | When sorting, below what size should data be concatenated and sorted in a single RecordBatch rather than sorted in batches and merged. | -| datafusion.execution.max_spill_file_size_bytes | 134217728 | Maximum size in bytes for individual spill files before rotating to a new file. When operators spill data to disk (e.g., RepartitionExec), they write multiple batches to the same file until this size limit is reached, then rotate to a new file. This reduces syscall overhead compared to one-file-per-batch while preventing files from growing too large. A larger value reduces file creation overhead but may hold more disk space. A smaller value creates more files but allows finer-grained space reclamation as files can be deleted once fully consumed. Now only `RepartitionExec` supports this spill file rotation feature, other spilling operators may create spill files larger than the limit. Default: 128 MB | -| datafusion.execution.meta_fetch_concurrency | 32 | Number of files to read in parallel when inferring schema and statistics | -| datafusion.execution.minimum_parallel_output_files | 4 | Guarantees a minimum level of output files running in parallel. RecordBatches will be distributed in round robin fashion to each parallel writer. Each writer is closed and a new file opened once soft_max_rows_per_output_file is reached. | -| datafusion.execution.soft_max_rows_per_output_file | 50000000 | Target number of rows in output files when writing multiple. This is a soft max, so it can be exceeded slightly. There also will be one file smaller than the limit if the total number of rows written is not roughly divisible by the soft max | -| datafusion.execution.max_buffered_batches_per_output_file | 2 | This is the maximum number of RecordBatches buffered for each output file being worked. Higher values can potentially give faster write performance at the cost of higher peak memory consumption | -| datafusion.execution.listing_table_ignore_subdirectory | true | Should sub directories be ignored when scanning directories for data files. Defaults to true (ignores subdirectories), consistent with Hive. Note that this setting does not affect reading partitioned tables (e.g. `/table/year=2021/month=01/data.parquet`). | -| datafusion.execution.listing_table_factory_infer_partitions | true | Should a `ListingTable` created through the `ListingTableFactory` infer table partitions from Hive compliant directories. Defaults to true (partition columns are inferred and will be represented in the table schema). | -| datafusion.execution.enable_recursive_ctes | true | Should DataFusion support recursive CTEs | -| datafusion.execution.split_file_groups_by_statistics | false | Attempt to eliminate sorts by packing & sorting files with non-overlapping statistics into the same file groups. Currently experimental | -| datafusion.execution.keep_partition_by_columns | false | Should DataFusion keep the columns used for partition_by in the output RecordBatches | -| datafusion.execution.skip_partial_aggregation_probe_ratio_threshold | 0.8 | Aggregation ratio (number of distinct groups / number of input rows) threshold for skipping partial aggregation. If the value is greater then partial aggregation will skip aggregation for further input | -| datafusion.execution.skip_partial_aggregation_probe_rows_threshold | 100000 | Number of input rows partial aggregation partition should process, before aggregation ratio check and trying to switch to skipping aggregation mode | -| datafusion.execution.use_row_number_estimates_to_optimize_partitioning | false | Should DataFusion use row number estimates at the input to decide whether increasing parallelism is beneficial or not. By default, only exact row numbers (not estimates) are used for this decision. Setting this flag to `true` will likely produce better plans. if the source of statistics is accurate. We plan to make this the default in the future. | -| datafusion.execution.enforce_batch_size_in_joins | false | Should DataFusion enforce batch size in joins or not. By default, DataFusion will not enforce batch size in joins. Enforcing batch size in joins can reduce memory usage when joining large tables with a highly-selective join filter, but is also slightly slower. | -| datafusion.execution.objectstore_writer_buffer_size | 10485760 | Size (bytes) of data buffer DataFusion uses when writing output files. This affects the size of the data chunks that are uploaded to remote object stores (e.g. AWS S3). If very large (>= 100 GiB) output files are being written, it may be necessary to increase this size to avoid errors from the remote end point. | -| datafusion.execution.enable_ansi_mode | false | Whether to enable ANSI SQL mode. The flag is experimental and relevant only for DataFusion Spark built-in functions When `enable_ansi_mode` is set to `true`, the query engine follows ANSI SQL semantics for expressions, casting, and error handling. This means: - **Strict type coercion rules:** implicit casts between incompatible types are disallowed. - **Standard SQL arithmetic behavior:** operations such as division by zero, numeric overflow, or invalid casts raise runtime errors rather than returning `NULL` or adjusted values. - **Consistent ANSI behavior** for string concatenation, comparisons, and `NULL` handling. When `enable_ansi_mode` is `false` (the default), the engine uses a more permissive, non-ANSI mode designed for user convenience and backward compatibility. In this mode: - Implicit casts between types are allowed (e.g., string to integer when possible). - Arithmetic operations are more lenient — for example, `abs()` on the minimum representable integer value returns the input value instead of raising overflow. - Division by zero or invalid casts may return `NULL` instead of failing. # Default `false` — ANSI SQL mode is disabled by default. | -| datafusion.optimizer.enable_distinct_aggregation_soft_limit | true | When set to true, the optimizer will push a limit operation into grouped aggregations which have no aggregate expressions, as a soft limit, emitting groups once the limit is reached, before all rows in the group are read. | -| datafusion.optimizer.enable_round_robin_repartition | true | When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores | -| datafusion.optimizer.enable_topk_aggregation | true | When set to true, the optimizer will attempt to perform limit operations during aggregations, if possible | -| datafusion.optimizer.enable_window_limits | true | When set to true, the optimizer will attempt to push limit operations past window functions, if possible | -| datafusion.optimizer.enable_topk_dynamic_filter_pushdown | true | When set to true, the optimizer will attempt to push down TopK dynamic filters into the file scan phase. | -| datafusion.optimizer.enable_join_dynamic_filter_pushdown | true | When set to true, the optimizer will attempt to push down Join dynamic filters into the file scan phase. | -| datafusion.optimizer.enable_aggregate_dynamic_filter_pushdown | true | When set to true, the optimizer will attempt to push down Aggregate dynamic filters into the file scan phase. | -| datafusion.optimizer.enable_dynamic_filter_pushdown | true | When set to true attempts to push down dynamic filters generated by operators (TopK, Join & Aggregate) into the file scan phase. For example, for a query such as `SELECT * FROM t ORDER BY timestamp DESC LIMIT 10`, the optimizer will attempt to push down the current top 10 timestamps that the TopK operator references into the file scans. This means that if we already have 10 timestamps in the year 2025 any files that only have timestamps in the year 2024 can be skipped / pruned at various stages in the scan. The config will suppress `enable_join_dynamic_filter_pushdown`, `enable_topk_dynamic_filter_pushdown` & `enable_aggregate_dynamic_filter_pushdown` So if you disable `enable_topk_dynamic_filter_pushdown`, then enable `enable_dynamic_filter_pushdown`, the `enable_topk_dynamic_filter_pushdown` will be overridden. | -| datafusion.optimizer.filter_null_join_keys | false | When set to true, the optimizer will insert filters before a join between a nullable and non-nullable column to filter out nulls on the nullable side. This filter can add additional overhead when the file format does not fully support predicate push down. | -| datafusion.optimizer.repartition_aggregations | true | Should DataFusion repartition data using the aggregate keys to execute aggregates in parallel using the provided `target_partitions` level | -| datafusion.optimizer.repartition_file_min_size | 10485760 | Minimum total files size in bytes to perform file scan repartitioning. | -| datafusion.optimizer.repartition_joins | true | Should DataFusion repartition data using the join keys to execute joins in parallel using the provided `target_partitions` level | -| datafusion.optimizer.allow_symmetric_joins_without_pruning | true | Should DataFusion allow symmetric hash joins for unbounded data sources even when its inputs do not have any ordering or filtering If the flag is not enabled, the SymmetricHashJoin operator will be unable to prune its internal buffers, resulting in certain join types - such as Full, Left, LeftAnti, LeftSemi, Right, RightAnti, and RightSemi - being produced only at the end of the execution. This is not typical in stream processing. Additionally, without proper design for long runner execution, all types of joins may encounter out-of-memory errors. | -| datafusion.optimizer.repartition_file_scans | true | When set to `true`, datasource partitions will be repartitioned to achieve maximum parallelism. This applies to both in-memory partitions and FileSource's file groups (1 group is 1 partition). For FileSources, only Parquet and CSV formats are currently supported. If set to `true` for a FileSource, all files will be repartitioned evenly (i.e., a single large file might be partitioned into smaller chunks) for parallel scanning. If set to `false` for a FileSource, different files will be read in parallel, but repartitioning won't happen within a single file. If set to `true` for an in-memory source, all memtable's partitions will have their batches repartitioned evenly to the desired number of `target_partitions`. Repartitioning can change the total number of partitions and batches per partition, but does not slice the initial record tables provided to the MemTable on creation. | -| datafusion.optimizer.preserve_file_partitions | 0 | Minimum number of distinct partition values required to group files by their Hive partition column values (enabling Hash partitioning declaration). How the option is used: - preserve_file_partitions=0: Disable it. - preserve_file_partitions=1: Always enable it. - preserve_file_partitions=N, actual file partitions=M: Only enable when M >= N. This threshold preserves I/O parallelism when file partitioning is below it. Note: This may reduce parallelism, rooting from the I/O level, if the number of distinct partitions is less than the target_partitions. | -| datafusion.optimizer.repartition_windows | true | Should DataFusion repartition data using the partitions keys to execute window functions in parallel using the provided `target_partitions` level | -| datafusion.optimizer.repartition_sorts | true | Should DataFusion execute sorts in a per-partition fashion and merge afterwards instead of coalescing first and sorting globally. With this flag is enabled, plans in the form below `text "SortExec: [a@0 ASC]", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", ` would turn into the plan below which performs better in multithreaded environments `text "SortPreservingMergeExec: [a@0 ASC]", " SortExec: [a@0 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", ` | -| datafusion.optimizer.subset_repartition_threshold | 4 | Partition count threshold for subset satisfaction optimization. When the current partition count is >= this threshold, DataFusion will skip repartitioning if the required partitioning expression is a subset of the current partition expression such as Hash(a) satisfies Hash(a, b). When the current partition count is < this threshold, DataFusion will repartition to increase parallelism even when subset satisfaction applies. Set to 0 to always repartition (disable subset satisfaction optimization). Set to a high value to always use subset satisfaction. Example (subset_repartition_threshold = 4): `text Hash([a]) satisfies Hash([a, b]) because (Hash([a, b]) is subset of Hash([a]) If current partitions (3) < threshold (4), repartition: AggregateExec: mode=FinalPartitioned, gby=[a, b], aggr=[SUM(x)] RepartitionExec: partitioning=Hash([a, b], 8), input_partitions=3 AggregateExec: mode=Partial, gby=[a, b], aggr=[SUM(x)] DataSourceExec: file_groups={...}, output_partitioning=Hash([a], 3) If current partitions (8) >= threshold (4), use subset satisfaction: AggregateExec: mode=SinglePartitioned, gby=[a, b], aggr=[SUM(x)] DataSourceExec: file_groups={...}, output_partitioning=Hash([a], 8) ` | -| datafusion.optimizer.prefer_existing_sort | false | When true, DataFusion will opportunistically remove sorts when the data is already sorted, (i.e. setting `preserve_order` to true on `RepartitionExec` and using `SortPreservingMergeExec`) When false, DataFusion will maximize plan parallelism using `RepartitionExec` even if this requires subsequently resorting data using a `SortExec`. | -| datafusion.optimizer.skip_failed_rules | false | When set to true, the logical plan optimizer will produce warning messages if any optimization rules produce errors and then proceed to the next rule. When set to false, any rules that produce errors will cause the query to fail | -| datafusion.optimizer.max_passes | 3 | Number of times that the optimizer will attempt to optimize the plan | -| datafusion.optimizer.top_down_join_key_reordering | true | When set to true, the physical plan optimizer will run a top down process to reorder the join keys | -| datafusion.optimizer.prefer_hash_join | true | When set to true, the physical plan optimizer will prefer HashJoin over SortMergeJoin. HashJoin can work more efficiently than SortMergeJoin but consumes more memory | -| datafusion.optimizer.enable_piecewise_merge_join | false | When set to true, piecewise merge join is enabled. PiecewiseMergeJoin is currently experimental. Physical planner will opt for PiecewiseMergeJoin when there is only one range filter. | -| datafusion.optimizer.hash_join_single_partition_threshold | 1048576 | The maximum estimated size in bytes for one input side of a HashJoin will be collected into a single partition | -| datafusion.optimizer.hash_join_single_partition_threshold_rows | 131072 | The maximum estimated size in rows for one input side of a HashJoin will be collected into a single partition | -| datafusion.optimizer.hash_join_inlist_pushdown_max_size | 131072 | Maximum size in bytes for the build side of a hash join to be pushed down as an InList expression for dynamic filtering. Build sides larger than this will use hash table lookups instead. Set to 0 to always use hash table lookups. InList pushdown can be more efficient for small build sides because it can result in better statistics pruning as well as use any bloom filters present on the scan side. InList expressions are also more transparent and easier to serialize over the network in distributed uses of DataFusion. On the other hand InList pushdown requires making a copy of the data and thus adds some overhead to the build side and uses more memory. This setting is per-partition, so we may end up using `hash_join_inlist_pushdown_max_size` \* `target_partitions` memory. The default is 128kB per partition. This should allow point lookup joins (e.g. joining on a unique primary key) to use InList pushdown in most cases but avoids excessive memory usage or overhead for larger joins. | -| datafusion.optimizer.hash_join_inlist_pushdown_max_distinct_values | 150 | Maximum number of distinct values (rows) in the build side of a hash join to be pushed down as an InList expression for dynamic filtering. Build sides with more rows than this will use hash table lookups instead. Set to 0 to always use hash table lookups. This provides an additional limit beyond `hash_join_inlist_pushdown_max_size` to prevent very large IN lists that might not provide much benefit over hash table lookups. This uses the deduplicated row count once the build side has been evaluated. The default is 150 values per partition. This is inspired by Trino's `max-filter-keys-per-column` setting. See: | -| datafusion.optimizer.default_filter_selectivity | 20 | The default filter selectivity used by Filter Statistics when an exact selectivity cannot be determined. Valid values are between 0 (no selectivity) and 100 (all rows are selected). | -| datafusion.optimizer.prefer_existing_union | false | When set to true, the optimizer will not attempt to convert Union to Interleave | -| datafusion.optimizer.expand_views_at_output | false | When set to true, if the returned type is a view type then the output will be coerced to a non-view. Coerces `Utf8View` to `LargeUtf8`, and `BinaryView` to `LargeBinary`. | -| datafusion.optimizer.enable_sort_pushdown | true | Enable sort pushdown optimization. When enabled, attempts to push sort requirements down to data sources that can natively handle them (e.g., by reversing file/row group read order). Returns **inexact ordering**: Sort operator is kept for correctness, but optimized input enables early termination for TopK queries (ORDER BY ... LIMIT N), providing significant speedup. Memory: No additional overhead (only changes read order). Future: Will add option to detect perfectly sorted data and eliminate Sort completely. Default: true | -| datafusion.optimizer.enable_leaf_expression_pushdown | true | When set to true, the optimizer will extract leaf expressions (such as `get_field`) from filter/sort/join nodes into projections closer to the leaf table scans, and push those projections down towards the leaf nodes. | -| datafusion.explain.logical_plan_only | false | When set to true, the explain statement will only print logical plans | -| datafusion.explain.physical_plan_only | false | When set to true, the explain statement will only print physical plans | -| datafusion.explain.show_statistics | false | When set to true, the explain statement will print operator statistics for physical plans | -| datafusion.explain.show_sizes | true | When set to true, the explain statement will print the partition sizes | -| datafusion.explain.show_schema | false | When set to true, the explain statement will print schema information | -| datafusion.explain.format | indent | Display format of explain. Default is "indent". When set to "tree", it will print the plan in a tree-rendered format. | -| datafusion.explain.tree_maximum_render_width | 240 | (format=tree only) Maximum total width of the rendered tree. When set to 0, the tree will have no width limit. | -| datafusion.explain.analyze_level | dev | Verbosity level for "EXPLAIN ANALYZE". Default is "dev" "summary" shows common metrics for high-level insights. "dev" provides deep operator-level introspection for developers. | -| datafusion.sql_parser.parse_float_as_decimal | false | When set to true, SQL parser will parse float as decimal type | -| datafusion.sql_parser.enable_ident_normalization | true | When set to true, SQL parser will normalize ident (convert ident to lowercase when not quoted) | -| datafusion.sql_parser.enable_options_value_normalization | false | When set to true, SQL parser will normalize options value (convert value to lowercase). Note that this option is ignored and will be removed in the future. All case-insensitive values are normalized automatically. | -| datafusion.sql_parser.dialect | generic | Configure the SQL dialect used by DataFusion's parser; supported values include: Generic, MySQL, PostgreSQL, Hive, SQLite, Snowflake, Redshift, MsSQL, ClickHouse, BigQuery, Ansi, DuckDB and Databricks. | -| datafusion.sql_parser.support_varchar_with_length | true | If true, permit lengths for `VARCHAR` such as `VARCHAR(20)`, but ignore the length. If false, error if a `VARCHAR` with a length is specified. The Arrow type system does not have a notion of maximum string length and thus DataFusion can not enforce such limits. | -| datafusion.sql_parser.map_string_types_to_utf8view | true | If true, string types (VARCHAR, CHAR, Text, and String) are mapped to `Utf8View` during SQL planning. If false, they are mapped to `Utf8`. Default is true. | -| datafusion.sql_parser.collect_spans | false | When set to true, the source locations relative to the original SQL query (i.e. [`Span`](https://docs.rs/sqlparser/latest/sqlparser/tokenizer/struct.Span.html)) will be collected and recorded in the logical plan nodes. | -| datafusion.sql_parser.recursion_limit | 50 | Specifies the recursion depth limit when parsing complex SQL Queries | -| datafusion.sql_parser.default_null_ordering | nulls_max | Specifies the default null ordering for query results. There are 4 options: - `nulls_max`: Nulls appear last in ascending order. - `nulls_min`: Nulls appear first in ascending order. - `nulls_first`: Nulls always be first in any order. - `nulls_last`: Nulls always be last in any order. By default, `nulls_max` is used to follow Postgres's behavior. postgres rule: | -| datafusion.format.safe | true | If set to `true` any formatting errors will be written to the output instead of being converted into a [`std::fmt::Error`] | -| datafusion.format.null | | Format string for nulls | -| datafusion.format.date_format | %Y-%m-%d | Date format for date arrays | -| datafusion.format.datetime_format | %Y-%m-%dT%H:%M:%S%.f | Format for DateTime arrays | -| datafusion.format.timestamp_format | %Y-%m-%dT%H:%M:%S%.f | Timestamp format for timestamp arrays | -| datafusion.format.timestamp_tz_format | NULL | Timestamp format for timestamp with timezone arrays. When `None`, ISO 8601 format is used. | -| datafusion.format.time_format | %H:%M:%S%.f | Time format for time arrays | -| datafusion.format.duration_format | pretty | Duration format. Can be either `"pretty"` or `"ISO8601"` | -| datafusion.format.types_info | false | Show types in visual representation batches | +| datafusion.execution.planning_concurrency | 0 | Fan-out during initial physical planning. This is mostly use to plan `UNION` children in parallel. Defaults to the number of CPU cores on the system | +| datafusion.execution.skip_physical_aggregate_schema_check | false | When set to true, skips verifying that the schema produced by planning the input of `LogicalPlan::Aggregate` exactly matches the schema of the input plan. When set to false, if the schema does not match exactly (including nullability and metadata), a planning error will be raised. This is used to workaround bugs in the planner that are now caught by the new schema verification step. | +| datafusion.execution.spill_compression | uncompressed | Sets the compression codec used when spilling data to disk. Since datafusion writes spill files using the Arrow IPC Stream format, only codecs supported by the Arrow IPC Stream Writer are allowed. Valid values are: uncompressed, lz4_frame, zstd. Note: lz4_frame offers faster (de)compression, but typically results in larger spill files. In contrast, zstd achieves higher compression ratios at the cost of slower (de)compression speed. | +| datafusion.execution.sort_spill_reservation_bytes | 10485760 | Specifies the reserved memory for each spillable sort operation to facilitate an in-memory merge. When a sort operation spills to disk, the in-memory data must be sorted and merged before being written to a file. This setting reserves a specific amount of memory for that in-memory sort/merge process. Note: This setting is irrelevant if the sort operation cannot spill (i.e., if there's no `DiskManager` configured). | +| datafusion.execution.sort_in_place_threshold_bytes | 1048576 | When sorting, below what size should data be concatenated and sorted in a single RecordBatch rather than sorted in batches and merged. | +| datafusion.execution.max_spill_file_size_bytes | 134217728 | Maximum size in bytes for individual spill files before rotating to a new file. When operators spill data to disk (e.g., RepartitionExec), they write multiple batches to the same file until this size limit is reached, then rotate to a new file. This reduces syscall overhead compared to one-file-per-batch while preventing files from growing too large. A larger value reduces file creation overhead but may hold more disk space. A smaller value creates more files but allows finer-grained space reclamation as files can be deleted once fully consumed. Now only `RepartitionExec` supports this spill file rotation feature, other spilling operators may create spill files larger than the limit. Default: 128 MB | +| datafusion.execution.meta_fetch_concurrency | 32 | Number of files to read in parallel when inferring schema and statistics | +| datafusion.execution.minimum_parallel_output_files | 4 | Guarantees a minimum level of output files running in parallel. RecordBatches will be distributed in round robin fashion to each parallel writer. Each writer is closed and a new file opened once soft_max_rows_per_output_file is reached. | +| datafusion.execution.soft_max_rows_per_output_file | 50000000 | Target number of rows in output files when writing multiple. This is a soft max, so it can be exceeded slightly. There also will be one file smaller than the limit if the total number of rows written is not roughly divisible by the soft max | +| datafusion.execution.max_buffered_batches_per_output_file | 2 | This is the maximum number of RecordBatches buffered for each output file being worked. Higher values can potentially give faster write performance at the cost of higher peak memory consumption | +| datafusion.execution.listing_table_ignore_subdirectory | true | Should sub directories be ignored when scanning directories for data files. Defaults to true (ignores subdirectories), consistent with Hive. Note that this setting does not affect reading partitioned tables (e.g. `/table/year=2021/month=01/data.parquet`). | +| datafusion.execution.listing_table_factory_infer_partitions | true | Should a `ListingTable` created through the `ListingTableFactory` infer table partitions from Hive compliant directories. Defaults to true (partition columns are inferred and will be represented in the table schema). | +| datafusion.execution.enable_recursive_ctes | true | Should DataFusion support recursive CTEs | +| datafusion.execution.split_file_groups_by_statistics | false | Attempt to eliminate sorts by packing & sorting files with non-overlapping statistics into the same file groups. Currently experimental | +| datafusion.execution.keep_partition_by_columns | false | Should DataFusion keep the columns used for partition_by in the output RecordBatches | +| datafusion.execution.skip_partial_aggregation_probe_ratio_threshold | 0.8 | Aggregation ratio (number of distinct groups / number of input rows) threshold for skipping partial aggregation. If the value is greater then partial aggregation will skip aggregation for further input | +| datafusion.execution.skip_partial_aggregation_probe_rows_threshold | 100000 | Number of input rows partial aggregation partition should process, before aggregation ratio check and trying to switch to skipping aggregation mode | +| datafusion.execution.use_row_number_estimates_to_optimize_partitioning | false | Should DataFusion use row number estimates at the input to decide whether increasing parallelism is beneficial or not. By default, only exact row numbers (not estimates) are used for this decision. Setting this flag to `true` will likely produce better plans. if the source of statistics is accurate. We plan to make this the default in the future. | +| datafusion.execution.enforce_batch_size_in_joins | false | Should DataFusion enforce batch size in joins or not. By default, DataFusion will not enforce batch size in joins. Enforcing batch size in joins can reduce memory usage when joining large tables with a highly-selective join filter, but is also slightly slower. | +| datafusion.execution.objectstore_writer_buffer_size | 10485760 | Size (bytes) of data buffer DataFusion uses when writing output files. This affects the size of the data chunks that are uploaded to remote object stores (e.g. AWS S3). If very large (>= 100 GiB) output files are being written, it may be necessary to increase this size to avoid errors from the remote end point. | +| datafusion.execution.enable_ansi_mode | false | Whether to enable ANSI SQL mode. The flag is experimental and relevant only for DataFusion Spark built-in functions When `enable_ansi_mode` is set to `true`, the query engine follows ANSI SQL semantics for expressions, casting, and error handling. This means: - **Strict type coercion rules:** implicit casts between incompatible types are disallowed. - **Standard SQL arithmetic behavior:** operations such as division by zero, numeric overflow, or invalid casts raise runtime errors rather than returning `NULL` or adjusted values. - **Consistent ANSI behavior** for string concatenation, comparisons, and `NULL` handling. When `enable_ansi_mode` is `false` (the default), the engine uses a more permissive, non-ANSI mode designed for user convenience and backward compatibility. In this mode: - Implicit casts between types are allowed (e.g., string to integer when possible). - Arithmetic operations are more lenient — for example, `abs()` on the minimum representable integer value returns the input value instead of raising overflow. - Division by zero or invalid casts may return `NULL` instead of failing. # Default `false` — ANSI SQL mode is disabled by default. | +| datafusion.optimizer.enable_distinct_aggregation_soft_limit | true | When set to true, the optimizer will push a limit operation into grouped aggregations which have no aggregate expressions, as a soft limit, emitting groups once the limit is reached, before all rows in the group are read. | +| datafusion.optimizer.enable_round_robin_repartition | true | When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores | +| datafusion.optimizer.enable_topk_aggregation | true | When set to true, the optimizer will attempt to perform limit operations during aggregations, if possible | +| datafusion.optimizer.enable_window_limits | true | When set to true, the optimizer will attempt to push limit operations past window functions, if possible | +| datafusion.optimizer.enable_topk_dynamic_filter_pushdown | true | When set to true, the optimizer will attempt to push down TopK dynamic filters into the file scan phase. | +| datafusion.optimizer.enable_join_dynamic_filter_pushdown | true | When set to true, the optimizer will attempt to push down Join dynamic filters into the file scan phase. | +| datafusion.optimizer.enable_aggregate_dynamic_filter_pushdown | true | When set to true, the optimizer will attempt to push down Aggregate dynamic filters into the file scan phase. | +| datafusion.optimizer.enable_dynamic_filter_pushdown | true | When set to true attempts to push down dynamic filters generated by operators (TopK, Join & Aggregate) into the file scan phase. For example, for a query such as `SELECT * FROM t ORDER BY timestamp DESC LIMIT 10`, the optimizer will attempt to push down the current top 10 timestamps that the TopK operator references into the file scans. This means that if we already have 10 timestamps in the year 2025 any files that only have timestamps in the year 2024 can be skipped / pruned at various stages in the scan. The config will suppress `enable_join_dynamic_filter_pushdown`, `enable_topk_dynamic_filter_pushdown` & `enable_aggregate_dynamic_filter_pushdown` So if you disable `enable_topk_dynamic_filter_pushdown`, then enable `enable_dynamic_filter_pushdown`, the `enable_topk_dynamic_filter_pushdown` will be overridden. | +| datafusion.optimizer.filter_null_join_keys | false | When set to true, the optimizer will insert filters before a join between a nullable and non-nullable column to filter out nulls on the nullable side. This filter can add additional overhead when the file format does not fully support predicate push down. | +| datafusion.optimizer.repartition_aggregations | true | Should DataFusion repartition data using the aggregate keys to execute aggregates in parallel using the provided `target_partitions` level | +| datafusion.optimizer.repartition_file_min_size | 10485760 | Minimum total files size in bytes to perform file scan repartitioning. | +| datafusion.optimizer.repartition_joins | true | Should DataFusion repartition data using the join keys to execute joins in parallel using the provided `target_partitions` level | +| datafusion.optimizer.allow_symmetric_joins_without_pruning | true | Should DataFusion allow symmetric hash joins for unbounded data sources even when its inputs do not have any ordering or filtering If the flag is not enabled, the SymmetricHashJoin operator will be unable to prune its internal buffers, resulting in certain join types - such as Full, Left, LeftAnti, LeftSemi, Right, RightAnti, and RightSemi - being produced only at the end of the execution. This is not typical in stream processing. Additionally, without proper design for long runner execution, all types of joins may encounter out-of-memory errors. | +| datafusion.optimizer.repartition_file_scans | true | When set to `true`, datasource partitions will be repartitioned to achieve maximum parallelism. This applies to both in-memory partitions and FileSource's file groups (1 group is 1 partition). For FileSources, only Parquet and CSV formats are currently supported. If set to `true` for a FileSource, all files will be repartitioned evenly (i.e., a single large file might be partitioned into smaller chunks) for parallel scanning. If set to `false` for a FileSource, different files will be read in parallel, but repartitioning won't happen within a single file. If set to `true` for an in-memory source, all memtable's partitions will have their batches repartitioned evenly to the desired number of `target_partitions`. Repartitioning can change the total number of partitions and batches per partition, but does not slice the initial record tables provided to the MemTable on creation. | +| datafusion.optimizer.preserve_file_partitions | 0 | Minimum number of distinct partition values required to group files by their Hive partition column values (enabling Hash partitioning declaration). How the option is used: - preserve_file_partitions=0: Disable it. - preserve_file_partitions=1: Always enable it. - preserve_file_partitions=N, actual file partitions=M: Only enable when M >= N. This threshold preserves I/O parallelism when file partitioning is below it. Note: This may reduce parallelism, rooting from the I/O level, if the number of distinct partitions is less than the target_partitions. | +| datafusion.optimizer.repartition_windows | true | Should DataFusion repartition data using the partitions keys to execute window functions in parallel using the provided `target_partitions` level | +| datafusion.optimizer.repartition_sorts | true | Should DataFusion execute sorts in a per-partition fashion and merge afterwards instead of coalescing first and sorting globally. With this flag is enabled, plans in the form below `text "SortExec: [a@0 ASC]", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", ` would turn into the plan below which performs better in multithreaded environments `text "SortPreservingMergeExec: [a@0 ASC]", " SortExec: [a@0 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", ` | +| datafusion.optimizer.subset_repartition_threshold | 4 | Partition count threshold for subset satisfaction optimization. When the current partition count is >= this threshold, DataFusion will skip repartitioning if the required partitioning expression is a subset of the current partition expression such as Hash(a) satisfies Hash(a, b). When the current partition count is < this threshold, DataFusion will repartition to increase parallelism even when subset satisfaction applies. Set to 0 to always repartition (disable subset satisfaction optimization). Set to a high value to always use subset satisfaction. Example (subset_repartition_threshold = 4): `text Hash([a]) satisfies Hash([a, b]) because (Hash([a, b]) is subset of Hash([a]) If current partitions (3) < threshold (4), repartition: AggregateExec: mode=FinalPartitioned, gby=[a, b], aggr=[SUM(x)] RepartitionExec: partitioning=Hash([a, b], 8), input_partitions=3 AggregateExec: mode=Partial, gby=[a, b], aggr=[SUM(x)] DataSourceExec: file_groups={...}, output_partitioning=Hash([a], 3) If current partitions (8) >= threshold (4), use subset satisfaction: AggregateExec: mode=SinglePartitioned, gby=[a, b], aggr=[SUM(x)] DataSourceExec: file_groups={...}, output_partitioning=Hash([a], 8) ` | +| datafusion.optimizer.prefer_existing_sort | false | When true, DataFusion will opportunistically remove sorts when the data is already sorted, (i.e. setting `preserve_order` to true on `RepartitionExec` and using `SortPreservingMergeExec`) When false, DataFusion will maximize plan parallelism using `RepartitionExec` even if this requires subsequently resorting data using a `SortExec`. | +| datafusion.optimizer.skip_failed_rules | false | When set to true, the logical plan optimizer will produce warning messages if any optimization rules produce errors and then proceed to the next rule. When set to false, any rules that produce errors will cause the query to fail | +| datafusion.optimizer.max_passes | 3 | Number of times that the optimizer will attempt to optimize the plan | +| datafusion.optimizer.top_down_join_key_reordering | true | When set to true, the physical plan optimizer will run a top down process to reorder the join keys | +| datafusion.optimizer.prefer_hash_join | true | When set to true, the physical plan optimizer will prefer HashJoin over SortMergeJoin. HashJoin can work more efficiently than SortMergeJoin but consumes more memory | +| datafusion.optimizer.enable_piecewise_merge_join | false | When set to true, piecewise merge join is enabled. PiecewiseMergeJoin is currently experimental. Physical planner will opt for PiecewiseMergeJoin when there is only one range filter. | +| datafusion.optimizer.hash_join_single_partition_threshold | 1048576 | The maximum estimated size in bytes for one input side of a HashJoin will be collected into a single partition | +| datafusion.optimizer.hash_join_single_partition_threshold_rows | 131072 | The maximum estimated size in rows for one input side of a HashJoin will be collected into a single partition | +| datafusion.optimizer.hash_join_inlist_pushdown_max_size | 131072 | Maximum size in bytes for the build side of a hash join to be pushed down as an InList expression for dynamic filtering. Build sides larger than this will use hash table lookups instead. Set to 0 to always use hash table lookups. InList pushdown can be more efficient for small build sides because it can result in better statistics pruning as well as use any bloom filters present on the scan side. InList expressions are also more transparent and easier to serialize over the network in distributed uses of DataFusion. On the other hand InList pushdown requires making a copy of the data and thus adds some overhead to the build side and uses more memory. This setting is per-partition, so we may end up using `hash_join_inlist_pushdown_max_size` \* `target_partitions` memory. The default is 128kB per partition. This should allow point lookup joins (e.g. joining on a unique primary key) to use InList pushdown in most cases but avoids excessive memory usage or overhead for larger joins. | +| datafusion.optimizer.hash_join_inlist_pushdown_max_distinct_values | 150 | Maximum number of distinct values (rows) in the build side of a hash join to be pushed down as an InList expression for dynamic filtering. Build sides with more rows than this will use hash table lookups instead. Set to 0 to always use hash table lookups. This provides an additional limit beyond `hash_join_inlist_pushdown_max_size` to prevent very large IN lists that might not provide much benefit over hash table lookups. This uses the deduplicated row count once the build side has been evaluated. The default is 150 values per partition. This is inspired by Trino's `max-filter-keys-per-column` setting. See: | +| datafusion.optimizer.default_filter_selectivity | 20 | The default filter selectivity used by Filter Statistics when an exact selectivity cannot be determined. Valid values are between 0 (no selectivity) and 100 (all rows are selected). | +| datafusion.optimizer.prefer_existing_union | false | When set to true, the optimizer will not attempt to convert Union to Interleave | +| datafusion.optimizer.expand_views_at_output | false | When set to true, if the returned type is a view type then the output will be coerced to a non-view. Coerces `Utf8View` to `LargeUtf8`, and `BinaryView` to `LargeBinary`. | +| datafusion.optimizer.enable_sort_pushdown | true | Enable sort pushdown optimization. When enabled, attempts to push sort requirements down to data sources that can natively handle them (e.g., by reversing file/row group read order). Returns **inexact ordering**: Sort operator is kept for correctness, but optimized input enables early termination for TopK queries (ORDER BY ... LIMIT N), providing significant speedup. Memory: No additional overhead (only changes read order). Future: Will add option to detect perfectly sorted data and eliminate Sort completely. Default: true | +| datafusion.optimizer.enable_leaf_expression_pushdown | true | When set to true, the optimizer will extract leaf expressions (such as `get_field`) from filter/sort/join nodes into projections closer to the leaf table scans, and push those projections down towards the leaf nodes. | +| datafusion.explain.logical_plan_only | false | When set to true, the explain statement will only print logical plans | +| datafusion.explain.physical_plan_only | false | When set to true, the explain statement will only print physical plans | +| datafusion.explain.show_statistics | false | When set to true, the explain statement will print operator statistics for physical plans | +| datafusion.explain.show_sizes | true | When set to true, the explain statement will print the partition sizes | +| datafusion.explain.show_schema | false | When set to true, the explain statement will print schema information | +| datafusion.explain.format | indent | Display format of explain. Default is "indent". When set to "tree", it will print the plan in a tree-rendered format. | +| datafusion.explain.tree_maximum_render_width | 240 | (format=tree only) Maximum total width of the rendered tree. When set to 0, the tree will have no width limit. | +| datafusion.explain.analyze_level | dev | Verbosity level for "EXPLAIN ANALYZE". Default is "dev" "summary" shows common metrics for high-level insights. "dev" provides deep operator-level introspection for developers. | +| datafusion.sql_parser.parse_float_as_decimal | false | When set to true, SQL parser will parse float as decimal type | +| datafusion.sql_parser.enable_ident_normalization | true | When set to true, SQL parser will normalize ident (convert ident to lowercase when not quoted) | +| datafusion.sql_parser.enable_options_value_normalization | false | When set to true, SQL parser will normalize options value (convert value to lowercase). Note that this option is ignored and will be removed in the future. All case-insensitive values are normalized automatically. | +| datafusion.sql_parser.dialect | generic | Configure the SQL dialect used by DataFusion's parser; supported values include: Generic, MySQL, PostgreSQL, Hive, SQLite, Snowflake, Redshift, MsSQL, ClickHouse, BigQuery, Ansi, DuckDB and Databricks. | +| datafusion.sql_parser.support_varchar_with_length | true | If true, permit lengths for `VARCHAR` such as `VARCHAR(20)`, but ignore the length. If false, error if a `VARCHAR` with a length is specified. The Arrow type system does not have a notion of maximum string length and thus DataFusion can not enforce such limits. | +| datafusion.sql_parser.map_string_types_to_utf8view | true | If true, string types (VARCHAR, CHAR, Text, and String) are mapped to `Utf8View` during SQL planning. If false, they are mapped to `Utf8`. Default is true. | +| datafusion.sql_parser.collect_spans | false | When set to true, the source locations relative to the original SQL query (i.e. [`Span`](https://docs.rs/sqlparser/latest/sqlparser/tokenizer/struct.Span.html)) will be collected and recorded in the logical plan nodes. | +| datafusion.sql_parser.recursion_limit | 50 | Specifies the recursion depth limit when parsing complex SQL Queries | +| datafusion.sql_parser.default_null_ordering | nulls_max | Specifies the default null ordering for query results. There are 4 options: - `nulls_max`: Nulls appear last in ascending order. - `nulls_min`: Nulls appear first in ascending order. - `nulls_first`: Nulls always be first in any order. - `nulls_last`: Nulls always be last in any order. By default, `nulls_max` is used to follow Postgres's behavior. postgres rule: | +| datafusion.format.safe | true | If set to `true` any formatting errors will be written to the output instead of being converted into a [`std::fmt::Error`] | +| datafusion.format.null | | Format string for nulls | +| datafusion.format.date_format | %Y-%m-%d | Date format for date arrays | +| datafusion.format.datetime_format | %Y-%m-%dT%H:%M:%S%.f | Format for DateTime arrays | +| datafusion.format.timestamp_format | %Y-%m-%dT%H:%M:%S%.f | Timestamp format for timestamp arrays | +| datafusion.format.timestamp_tz_format | NULL | Timestamp format for timestamp with timezone arrays. When `None`, ISO 8601 format is used. | +| datafusion.format.time_format | %H:%M:%S%.f | Time format for time arrays | +| datafusion.format.duration_format | pretty | Duration format. Can be either `"pretty"` or `"ISO8601"` | +| datafusion.format.types_info | false | Show types in visual representation batches | # Runtime Configuration Settings From 35f0f4e1aba4bf441357fe849e79e2cb2f913712 Mon Sep 17 00:00:00 2001 From: Subham Singhal Date: Sun, 22 Feb 2026 16:15:37 +0530 Subject: [PATCH 07/15] Fix formatting --- datafusion/physical-expr/src/utils/mod.rs | 19 +- .../test_files/information_schema.slt | 2 +- docs/source/user-guide/configs.md | 258 +++++++++--------- 3 files changed, 133 insertions(+), 146 deletions(-) diff --git a/datafusion/physical-expr/src/utils/mod.rs b/datafusion/physical-expr/src/utils/mod.rs index 9abca794495a6..36538ab0ec253 100644 --- a/datafusion/physical-expr/src/utils/mod.rs +++ b/datafusion/physical-expr/src/utils/mod.rs @@ -275,13 +275,6 @@ fn get_field_id(field: &arrow::datatypes::Field) -> Option { } /// Find field index by field ID with fallback to name-based matching -/// -/// # Limitations -/// -/// TODO: Currently only supports flat schemas. For nested schemas, this function -/// would need to accept a field path (e.g., ["address", "city"]) and return -/// a path of indices. This requires matching nested field IDs at each level -/// of the schema hierarchy. fn find_field_index( column_name: &str, source_schema: &Schema, @@ -293,8 +286,6 @@ fn find_field_index( // Check if field has a field ID if let Some(source_field_id) = get_field_id(source_field) { // Search target schema for matching field ID - // TODO: For nested schemas, this needs to recursively match field IDs - // through the struct hierarchy for (idx, target_field) in target_schema.fields().iter().enumerate() { if let Some(target_field_id) = get_field_id(target_field) && source_field_id == target_field_id @@ -322,13 +313,9 @@ fn find_field_index( /// /// # Limitations /// -/// TODO: Currently only supports flat schemas (top-level columns). Nested field -/// references (e.g., "address.city") are not yet supported. Supporting nested -/// fields would require: -/// - Path-based field ID matching through struct hierarchies -/// - Recursive traversal of both expression tree and schema tree -/// - Updates to Column representation to track nested paths -/// +/// Currently only supports flat schemas (top-level columns). Nested field +/// references (e.g., "address.city") are not yet supported. +/// For nested schema see: () /// # Errors /// /// This function will return an error if any column in the expression cannot be found diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index ba570ba192b0e..67a66be0a54af 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -383,7 +383,7 @@ datafusion.execution.parquet.dictionary_enabled true (writing) Sets if dictionar datafusion.execution.parquet.dictionary_page_size_limit 1048576 (writing) Sets best effort maximum dictionary page size, in bytes datafusion.execution.parquet.enable_page_index true (reading) If true, reads the Parquet data page level metadata (the Page Index), if present, to reduce the I/O and number of rows decoded. datafusion.execution.parquet.encoding NULL (writing) Sets default encoding for any column. Valid values are: plain, plain_dictionary, rle, bit_packed, delta_binary_packed, delta_length_byte_array, delta_byte_array, rle_dictionary, and byte_stream_split. These values are not case sensitive. If NULL, uses default parquet writer setting -datafusion.execution.parquet.field_id_read_enabled false (reading) If true, use Parquet field IDs for column resolution instead of column names. This enables schema evolution with renamed/reordered columns. When field IDs are unavailable, falls back to name-based matching. +datafusion.execution.parquet.field_id_enabled false (reading) If true, use Parquet field IDs for column resolution instead of column names. This enables schema evolution with renamed/reordered columns. When field IDs are unavailable, falls back to name-based matching. datafusion.execution.parquet.force_filter_selections false (reading) Force the use of RowSelections for filter results, when pushdown_filters is enabled. If false, the reader will automatically choose between a RowSelection and a Bitmap based on the number and pattern of selected rows. datafusion.execution.parquet.max_predicate_cache_size NULL (reading) The maximum predicate cache size, in bytes. When `pushdown_filters` is enabled, sets the maximum memory used to cache the results of predicate evaluation between filter evaluation and output generation. Decreasing this value will reduce memory usage, but may increase IO and CPU usage. None means use the default parquet reader setting. 0 means no caching. datafusion.execution.parquet.max_row_group_size 1048576 (writing) Target maximum number of rows in each row group (defaults to 1M rows). Writing larger row groups requires more memory to write, but can get better compression and be faster to read. diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index e925f5f689f66..5b21ac9d56ea4 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -63,136 +63,136 @@ SET datafusion.execution.target_partitions = '1'; The following configuration settings are available: -| key | default | description | -| ------------------------------------------------------------------ | ------------------------- | -------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | -| datafusion.catalog.create_default_catalog_and_schema | true | Whether the default catalog and schema should be created automatically. | -| datafusion.catalog.default_catalog | datafusion | The default catalog name - this impacts what SQL queries use if not specified | -| datafusion.catalog.default_schema | public | The default schema name - this impacts what SQL queries use if not specified | -| datafusion.catalog.information_schema | false | Should DataFusion provide access to `information_schema` virtual tables for displaying schema information | -| datafusion.catalog.location | NULL | Location scanned to load tables for `default` schema | -| datafusion.catalog.format | NULL | Type of `TableProvider` to use when loading `default` schema | -| datafusion.catalog.has_header | true | Default value for `format.has_header` for `CREATE EXTERNAL TABLE` if not specified explicitly in the statement. | -| datafusion.catalog.newlines_in_values | false | Specifies whether newlines in (quoted) CSV values are supported. This is the default value for `format.newlines_in_values` for `CREATE EXTERNAL TABLE` if not specified explicitly in the statement. Parsing newlines in quoted values may be affected by execution behaviour such as parallel file scanning. Setting this to `true` ensures that newlines in values are parsed successfully, which may reduce performance. | -| datafusion.execution.batch_size | 8192 | Default batch size while creating new batches, it's especially useful for buffer-in-memory batches since creating tiny batches would result in too much metadata memory consumption | -| datafusion.execution.perfect_hash_join_small_build_threshold | 1024 | A perfect hash join (see `HashJoinExec` for more details) will be considered if the range of keys (max - min) on the build side is < this threshold. This provides a fast path for joins with very small key ranges, bypassing the density check. Currently only supports cases where build_side.num_rows() < u32::MAX. Support for build_side.num_rows() >= u32::MAX will be added in the future. | -| datafusion.execution.perfect_hash_join_min_key_density | 0.15 | The minimum required density of join keys on the build side to consider a perfect hash join (see `HashJoinExec` for more details). Density is calculated as: `(number of rows) / (max_key - min_key + 1)`. A perfect hash join may be used if the actual key density > this value. Currently only supports cases where build_side.num_rows() < u32::MAX. Support for build_side.num_rows() >= u32::MAX will be added in the future. | -| datafusion.execution.coalesce_batches | true | When set to true, record batches will be examined between each operator and small batches will be coalesced into larger batches. This is helpful when there are highly selective filters or joins that could produce tiny output batches. The target batch size is determined by the configuration setting | -| datafusion.execution.collect_statistics | true | Should DataFusion collect statistics when first creating a table. Has no effect after the table is created. Applies to the default `ListingTableProvider` in DataFusion. Defaults to true. | -| datafusion.execution.target_partitions | 0 | Number of partitions for query execution. Increasing partitions can increase concurrency. Defaults to the number of CPU cores on the system | -| datafusion.execution.time_zone | NULL | The default time zone Some functions, e.g. `now` return timestamps in this time zone | -| datafusion.execution.parquet.enable_page_index | true | (reading) If true, reads the Parquet data page level metadata (the Page Index), if present, to reduce the I/O and number of rows decoded. | -| datafusion.execution.parquet.pruning | true | (reading) If true, the parquet reader attempts to skip entire row groups based on the predicate in the query and the metadata (min/max values) stored in the parquet file | -| datafusion.execution.parquet.skip_metadata | true | (reading) If true, the parquet reader skip the optional embedded metadata that may be in the file Schema. This setting can help avoid schema conflicts when querying multiple parquet files with schemas containing compatible types but different metadata | -| datafusion.execution.parquet.metadata_size_hint | 524288 | (reading) If specified, the parquet reader will try and fetch the last `size_hint` bytes of the parquet file optimistically. If not specified, two reads are required: One read to fetch the 8-byte parquet footer and another to fetch the metadata length encoded in the footer Default setting to 512 KiB, which should be sufficient for most parquet files, it can reduce one I/O operation per parquet file. If the metadata is larger than the hint, two reads will still be performed. | -| datafusion.execution.parquet.pushdown_filters | false | (reading) If true, filter expressions are be applied during the parquet decoding operation to reduce the number of rows decoded. This optimization is sometimes called "late materialization". | -| datafusion.execution.parquet.reorder_filters | false | (reading) If true, filter expressions evaluated during the parquet decoding operation will be reordered heuristically to minimize the cost of evaluation. If false, the filters are applied in the same order as written in the query | -| datafusion.execution.parquet.force_filter_selections | false | (reading) Force the use of RowSelections for filter results, when pushdown_filters is enabled. If false, the reader will automatically choose between a RowSelection and a Bitmap based on the number and pattern of selected rows. | -| datafusion.execution.parquet.schema_force_view_types | true | (reading) If true, parquet reader will read columns of `Utf8/Utf8Large` with `Utf8View`, and `Binary/BinaryLarge` with `BinaryView`. | -| datafusion.execution.parquet.binary_as_string | false | (reading) If true, parquet reader will read columns of `Binary/LargeBinary` with `Utf8`, and `BinaryView` with `Utf8View`. Parquet files generated by some legacy writers do not correctly set the UTF8 flag for strings, causing string columns to be loaded as BLOB instead. | -| datafusion.execution.parquet.coerce_int96 | NULL | (reading) If true, parquet reader will read columns of physical type int96 as originating from a different resolution than nanosecond. This is useful for reading data from systems like Spark which stores microsecond resolution timestamps in an int96 allowing it to write values with a larger date range than 64-bit timestamps with nanosecond resolution. | -| datafusion.execution.parquet.bloom_filter_on_read | true | (reading) Use any available bloom filters when reading parquet files | -| datafusion.execution.parquet.max_predicate_cache_size | NULL | (reading) The maximum predicate cache size, in bytes. When `pushdown_filters` is enabled, sets the maximum memory used to cache the results of predicate evaluation between filter evaluation and output generation. Decreasing this value will reduce memory usage, but may increase IO and CPU usage. None means use the default parquet reader setting. 0 means no caching. | -| datafusion.execution.parquet.field_id_enabled | false | (reading) If true, use Parquet field IDs for column resolution instead of column names. This enables schema evolution with renamed/reordered columns. When field IDs are unavailable, falls back to name-based matching. | -| datafusion.execution.parquet.data_pagesize_limit | 1048576 | (writing) Sets best effort maximum size of data page in bytes | -| datafusion.execution.parquet.write_batch_size | 1024 | (writing) Sets write_batch_size in rows | -| datafusion.execution.parquet.writer_version | 1.0 | (writing) Sets parquet writer version valid values are "1.0" and "2.0" | -| datafusion.execution.parquet.skip_arrow_metadata | false | (writing) Skip encoding the embedded arrow metadata in the KV_meta This is analogous to the `ArrowWriterOptions::with_skip_arrow_metadata`. Refer to | -| datafusion.execution.parquet.compression | zstd(3) | (writing) Sets default parquet compression codec. Valid values are: uncompressed, snappy, gzip(level), brotli(level), lz4, zstd(level), and lz4_raw. These values are not case sensitive. If NULL, uses default parquet writer setting Note that this default setting is not the same as the default parquet writer setting. | -| datafusion.execution.parquet.dictionary_enabled | true | (writing) Sets if dictionary encoding is enabled. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.dictionary_page_size_limit | 1048576 | (writing) Sets best effort maximum dictionary page size, in bytes | -| datafusion.execution.parquet.statistics_enabled | page | (writing) Sets if statistics are enabled for any column Valid values are: "none", "chunk", and "page" These values are not case sensitive. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.max_row_group_size | 1048576 | (writing) Target maximum number of rows in each row group (defaults to 1M rows). Writing larger row groups requires more memory to write, but can get better compression and be faster to read. | -| datafusion.execution.parquet.created_by | datafusion version 52.1.0 | (writing) Sets "created by" property | -| datafusion.execution.parquet.column_index_truncate_length | 64 | (writing) Sets column index truncate length | -| datafusion.execution.parquet.statistics_truncate_length | 64 | (writing) Sets statistics truncate length. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.data_page_row_count_limit | 20000 | (writing) Sets best effort maximum number of rows in data page | -| datafusion.execution.parquet.encoding | NULL | (writing) Sets default encoding for any column. Valid values are: plain, plain_dictionary, rle, bit_packed, delta_binary_packed, delta_length_byte_array, delta_byte_array, rle_dictionary, and byte_stream_split. These values are not case sensitive. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.bloom_filter_on_write | false | (writing) Write bloom filters for all columns when creating parquet files | -| datafusion.execution.parquet.bloom_filter_fpp | NULL | (writing) Sets bloom filter false positive probability. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.bloom_filter_ndv | NULL | (writing) Sets bloom filter number of distinct values. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.allow_single_file_parallelism | true | (writing) Controls whether DataFusion will attempt to speed up writing parquet files by serializing them in parallel. Each column in each row group in each output file are serialized in parallel leveraging a maximum possible core count of n_files*n_row_groups*n_columns. | -| datafusion.execution.parquet.maximum_parallel_row_group_writers | 1 | (writing) By default parallel parquet writer is tuned for minimum memory usage in a streaming execution plan. You may see a performance benefit when writing large parquet files by increasing maximum_parallel_row_group_writers and maximum_buffered_record_batches_per_stream if your system has idle cores and can tolerate additional memory usage. Boosting these values is likely worthwhile when writing out already in-memory data, such as from a cached data frame. | +| key | default | description | +| ----------------------------------------------------------------------- | ------------------------- | -------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | +| datafusion.catalog.create_default_catalog_and_schema | true | Whether the default catalog and schema should be created automatically. | +| datafusion.catalog.default_catalog | datafusion | The default catalog name - this impacts what SQL queries use if not specified | +| datafusion.catalog.default_schema | public | The default schema name - this impacts what SQL queries use if not specified | +| datafusion.catalog.information_schema | false | Should DataFusion provide access to `information_schema` virtual tables for displaying schema information | +| datafusion.catalog.location | NULL | Location scanned to load tables for `default` schema | +| datafusion.catalog.format | NULL | Type of `TableProvider` to use when loading `default` schema | +| datafusion.catalog.has_header | true | Default value for `format.has_header` for `CREATE EXTERNAL TABLE` if not specified explicitly in the statement. | +| datafusion.catalog.newlines_in_values | false | Specifies whether newlines in (quoted) CSV values are supported. This is the default value for `format.newlines_in_values` for `CREATE EXTERNAL TABLE` if not specified explicitly in the statement. Parsing newlines in quoted values may be affected by execution behaviour such as parallel file scanning. Setting this to `true` ensures that newlines in values are parsed successfully, which may reduce performance. | +| datafusion.execution.batch_size | 8192 | Default batch size while creating new batches, it's especially useful for buffer-in-memory batches since creating tiny batches would result in too much metadata memory consumption | +| datafusion.execution.perfect_hash_join_small_build_threshold | 1024 | A perfect hash join (see `HashJoinExec` for more details) will be considered if the range of keys (max - min) on the build side is < this threshold. This provides a fast path for joins with very small key ranges, bypassing the density check. Currently only supports cases where build_side.num_rows() < u32::MAX. Support for build_side.num_rows() >= u32::MAX will be added in the future. | +| datafusion.execution.perfect_hash_join_min_key_density | 0.15 | The minimum required density of join keys on the build side to consider a perfect hash join (see `HashJoinExec` for more details). Density is calculated as: `(number of rows) / (max_key - min_key + 1)`. A perfect hash join may be used if the actual key density > this value. Currently only supports cases where build_side.num_rows() < u32::MAX. Support for build_side.num_rows() >= u32::MAX will be added in the future. | +| datafusion.execution.coalesce_batches | true | When set to true, record batches will be examined between each operator and small batches will be coalesced into larger batches. This is helpful when there are highly selective filters or joins that could produce tiny output batches. The target batch size is determined by the configuration setting | +| datafusion.execution.collect_statistics | true | Should DataFusion collect statistics when first creating a table. Has no effect after the table is created. Applies to the default `ListingTableProvider` in DataFusion. Defaults to true. | +| datafusion.execution.target_partitions | 0 | Number of partitions for query execution. Increasing partitions can increase concurrency. Defaults to the number of CPU cores on the system | +| datafusion.execution.time_zone | NULL | The default time zone Some functions, e.g. `now` return timestamps in this time zone | +| datafusion.execution.parquet.enable_page_index | true | (reading) If true, reads the Parquet data page level metadata (the Page Index), if present, to reduce the I/O and number of rows decoded. | +| datafusion.execution.parquet.pruning | true | (reading) If true, the parquet reader attempts to skip entire row groups based on the predicate in the query and the metadata (min/max values) stored in the parquet file | +| datafusion.execution.parquet.skip_metadata | true | (reading) If true, the parquet reader skip the optional embedded metadata that may be in the file Schema. This setting can help avoid schema conflicts when querying multiple parquet files with schemas containing compatible types but different metadata | +| datafusion.execution.parquet.metadata_size_hint | 524288 | (reading) If specified, the parquet reader will try and fetch the last `size_hint` bytes of the parquet file optimistically. If not specified, two reads are required: One read to fetch the 8-byte parquet footer and another to fetch the metadata length encoded in the footer Default setting to 512 KiB, which should be sufficient for most parquet files, it can reduce one I/O operation per parquet file. If the metadata is larger than the hint, two reads will still be performed. | +| datafusion.execution.parquet.pushdown_filters | false | (reading) If true, filter expressions are be applied during the parquet decoding operation to reduce the number of rows decoded. This optimization is sometimes called "late materialization". | +| datafusion.execution.parquet.reorder_filters | false | (reading) If true, filter expressions evaluated during the parquet decoding operation will be reordered heuristically to minimize the cost of evaluation. If false, the filters are applied in the same order as written in the query | +| datafusion.execution.parquet.force_filter_selections | false | (reading) Force the use of RowSelections for filter results, when pushdown_filters is enabled. If false, the reader will automatically choose between a RowSelection and a Bitmap based on the number and pattern of selected rows. | +| datafusion.execution.parquet.schema_force_view_types | true | (reading) If true, parquet reader will read columns of `Utf8/Utf8Large` with `Utf8View`, and `Binary/BinaryLarge` with `BinaryView`. | +| datafusion.execution.parquet.binary_as_string | false | (reading) If true, parquet reader will read columns of `Binary/LargeBinary` with `Utf8`, and `BinaryView` with `Utf8View`. Parquet files generated by some legacy writers do not correctly set the UTF8 flag for strings, causing string columns to be loaded as BLOB instead. | +| datafusion.execution.parquet.coerce_int96 | NULL | (reading) If true, parquet reader will read columns of physical type int96 as originating from a different resolution than nanosecond. This is useful for reading data from systems like Spark which stores microsecond resolution timestamps in an int96 allowing it to write values with a larger date range than 64-bit timestamps with nanosecond resolution. | +| datafusion.execution.parquet.bloom_filter_on_read | true | (reading) Use any available bloom filters when reading parquet files | +| datafusion.execution.parquet.max_predicate_cache_size | NULL | (reading) The maximum predicate cache size, in bytes. When `pushdown_filters` is enabled, sets the maximum memory used to cache the results of predicate evaluation between filter evaluation and output generation. Decreasing this value will reduce memory usage, but may increase IO and CPU usage. None means use the default parquet reader setting. 0 means no caching. | +| datafusion.execution.parquet.field_id_enabled | false | (reading) If true, use Parquet field IDs for column resolution instead of column names. This enables schema evolution with renamed/reordered columns. When field IDs are unavailable, falls back to name-based matching. | +| datafusion.execution.parquet.data_pagesize_limit | 1048576 | (writing) Sets best effort maximum size of data page in bytes | +| datafusion.execution.parquet.write_batch_size | 1024 | (writing) Sets write_batch_size in rows | +| datafusion.execution.parquet.writer_version | 1.0 | (writing) Sets parquet writer version valid values are "1.0" and "2.0" | +| datafusion.execution.parquet.skip_arrow_metadata | false | (writing) Skip encoding the embedded arrow metadata in the KV_meta This is analogous to the `ArrowWriterOptions::with_skip_arrow_metadata`. Refer to | +| datafusion.execution.parquet.compression | zstd(3) | (writing) Sets default parquet compression codec. Valid values are: uncompressed, snappy, gzip(level), brotli(level), lz4, zstd(level), and lz4_raw. These values are not case sensitive. If NULL, uses default parquet writer setting Note that this default setting is not the same as the default parquet writer setting. | +| datafusion.execution.parquet.dictionary_enabled | true | (writing) Sets if dictionary encoding is enabled. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.dictionary_page_size_limit | 1048576 | (writing) Sets best effort maximum dictionary page size, in bytes | +| datafusion.execution.parquet.statistics_enabled | page | (writing) Sets if statistics are enabled for any column Valid values are: "none", "chunk", and "page" These values are not case sensitive. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.max_row_group_size | 1048576 | (writing) Target maximum number of rows in each row group (defaults to 1M rows). Writing larger row groups requires more memory to write, but can get better compression and be faster to read. | +| datafusion.execution.parquet.created_by | datafusion version 52.1.0 | (writing) Sets "created by" property | +| datafusion.execution.parquet.column_index_truncate_length | 64 | (writing) Sets column index truncate length | +| datafusion.execution.parquet.statistics_truncate_length | 64 | (writing) Sets statistics truncate length. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.data_page_row_count_limit | 20000 | (writing) Sets best effort maximum number of rows in data page | +| datafusion.execution.parquet.encoding | NULL | (writing) Sets default encoding for any column. Valid values are: plain, plain_dictionary, rle, bit_packed, delta_binary_packed, delta_length_byte_array, delta_byte_array, rle_dictionary, and byte_stream_split. These values are not case sensitive. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.bloom_filter_on_write | false | (writing) Write bloom filters for all columns when creating parquet files | +| datafusion.execution.parquet.bloom_filter_fpp | NULL | (writing) Sets bloom filter false positive probability. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.bloom_filter_ndv | NULL | (writing) Sets bloom filter number of distinct values. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.allow_single_file_parallelism | true | (writing) Controls whether DataFusion will attempt to speed up writing parquet files by serializing them in parallel. Each column in each row group in each output file are serialized in parallel leveraging a maximum possible core count of n_files*n_row_groups*n_columns. | +| datafusion.execution.parquet.maximum_parallel_row_group_writers | 1 | (writing) By default parallel parquet writer is tuned for minimum memory usage in a streaming execution plan. You may see a performance benefit when writing large parquet files by increasing maximum_parallel_row_group_writers and maximum_buffered_record_batches_per_stream if your system has idle cores and can tolerate additional memory usage. Boosting these values is likely worthwhile when writing out already in-memory data, such as from a cached data frame. | | datafusion.execution.parquet.maximum_buffered_record_batches_per_stream | 2 | (writing) By default parallel parquet writer is tuned for minimum memory usage in a streaming execution plan. You may see a performance benefit when writing large parquet files by increasing maximum_parallel_row_group_writers and maximum_buffered_record_batches_per_stream if your system has idle cores and can tolerate additional memory usage. Boosting these values is likely worthwhile when writing out already in-memory data, such as from a cached data frame. | -| datafusion.execution.planning_concurrency | 0 | Fan-out during initial physical planning. This is mostly use to plan `UNION` children in parallel. Defaults to the number of CPU cores on the system | -| datafusion.execution.skip_physical_aggregate_schema_check | false | When set to true, skips verifying that the schema produced by planning the input of `LogicalPlan::Aggregate` exactly matches the schema of the input plan. When set to false, if the schema does not match exactly (including nullability and metadata), a planning error will be raised. This is used to workaround bugs in the planner that are now caught by the new schema verification step. | -| datafusion.execution.spill_compression | uncompressed | Sets the compression codec used when spilling data to disk. Since datafusion writes spill files using the Arrow IPC Stream format, only codecs supported by the Arrow IPC Stream Writer are allowed. Valid values are: uncompressed, lz4_frame, zstd. Note: lz4_frame offers faster (de)compression, but typically results in larger spill files. In contrast, zstd achieves higher compression ratios at the cost of slower (de)compression speed. | -| datafusion.execution.sort_spill_reservation_bytes | 10485760 | Specifies the reserved memory for each spillable sort operation to facilitate an in-memory merge. When a sort operation spills to disk, the in-memory data must be sorted and merged before being written to a file. This setting reserves a specific amount of memory for that in-memory sort/merge process. Note: This setting is irrelevant if the sort operation cannot spill (i.e., if there's no `DiskManager` configured). | -| datafusion.execution.sort_in_place_threshold_bytes | 1048576 | When sorting, below what size should data be concatenated and sorted in a single RecordBatch rather than sorted in batches and merged. | -| datafusion.execution.max_spill_file_size_bytes | 134217728 | Maximum size in bytes for individual spill files before rotating to a new file. When operators spill data to disk (e.g., RepartitionExec), they write multiple batches to the same file until this size limit is reached, then rotate to a new file. This reduces syscall overhead compared to one-file-per-batch while preventing files from growing too large. A larger value reduces file creation overhead but may hold more disk space. A smaller value creates more files but allows finer-grained space reclamation as files can be deleted once fully consumed. Now only `RepartitionExec` supports this spill file rotation feature, other spilling operators may create spill files larger than the limit. Default: 128 MB | -| datafusion.execution.meta_fetch_concurrency | 32 | Number of files to read in parallel when inferring schema and statistics | -| datafusion.execution.minimum_parallel_output_files | 4 | Guarantees a minimum level of output files running in parallel. RecordBatches will be distributed in round robin fashion to each parallel writer. Each writer is closed and a new file opened once soft_max_rows_per_output_file is reached. | -| datafusion.execution.soft_max_rows_per_output_file | 50000000 | Target number of rows in output files when writing multiple. This is a soft max, so it can be exceeded slightly. There also will be one file smaller than the limit if the total number of rows written is not roughly divisible by the soft max | -| datafusion.execution.max_buffered_batches_per_output_file | 2 | This is the maximum number of RecordBatches buffered for each output file being worked. Higher values can potentially give faster write performance at the cost of higher peak memory consumption | -| datafusion.execution.listing_table_ignore_subdirectory | true | Should sub directories be ignored when scanning directories for data files. Defaults to true (ignores subdirectories), consistent with Hive. Note that this setting does not affect reading partitioned tables (e.g. `/table/year=2021/month=01/data.parquet`). | -| datafusion.execution.listing_table_factory_infer_partitions | true | Should a `ListingTable` created through the `ListingTableFactory` infer table partitions from Hive compliant directories. Defaults to true (partition columns are inferred and will be represented in the table schema). | -| datafusion.execution.enable_recursive_ctes | true | Should DataFusion support recursive CTEs | -| datafusion.execution.split_file_groups_by_statistics | false | Attempt to eliminate sorts by packing & sorting files with non-overlapping statistics into the same file groups. Currently experimental | -| datafusion.execution.keep_partition_by_columns | false | Should DataFusion keep the columns used for partition_by in the output RecordBatches | -| datafusion.execution.skip_partial_aggregation_probe_ratio_threshold | 0.8 | Aggregation ratio (number of distinct groups / number of input rows) threshold for skipping partial aggregation. If the value is greater then partial aggregation will skip aggregation for further input | -| datafusion.execution.skip_partial_aggregation_probe_rows_threshold | 100000 | Number of input rows partial aggregation partition should process, before aggregation ratio check and trying to switch to skipping aggregation mode | -| datafusion.execution.use_row_number_estimates_to_optimize_partitioning | false | Should DataFusion use row number estimates at the input to decide whether increasing parallelism is beneficial or not. By default, only exact row numbers (not estimates) are used for this decision. Setting this flag to `true` will likely produce better plans. if the source of statistics is accurate. We plan to make this the default in the future. | -| datafusion.execution.enforce_batch_size_in_joins | false | Should DataFusion enforce batch size in joins or not. By default, DataFusion will not enforce batch size in joins. Enforcing batch size in joins can reduce memory usage when joining large tables with a highly-selective join filter, but is also slightly slower. | -| datafusion.execution.objectstore_writer_buffer_size | 10485760 | Size (bytes) of data buffer DataFusion uses when writing output files. This affects the size of the data chunks that are uploaded to remote object stores (e.g. AWS S3). If very large (>= 100 GiB) output files are being written, it may be necessary to increase this size to avoid errors from the remote end point. | -| datafusion.execution.enable_ansi_mode | false | Whether to enable ANSI SQL mode. The flag is experimental and relevant only for DataFusion Spark built-in functions When `enable_ansi_mode` is set to `true`, the query engine follows ANSI SQL semantics for expressions, casting, and error handling. This means: - **Strict type coercion rules:** implicit casts between incompatible types are disallowed. - **Standard SQL arithmetic behavior:** operations such as division by zero, numeric overflow, or invalid casts raise runtime errors rather than returning `NULL` or adjusted values. - **Consistent ANSI behavior** for string concatenation, comparisons, and `NULL` handling. When `enable_ansi_mode` is `false` (the default), the engine uses a more permissive, non-ANSI mode designed for user convenience and backward compatibility. In this mode: - Implicit casts between types are allowed (e.g., string to integer when possible). - Arithmetic operations are more lenient — for example, `abs()` on the minimum representable integer value returns the input value instead of raising overflow. - Division by zero or invalid casts may return `NULL` instead of failing. # Default `false` — ANSI SQL mode is disabled by default. | -| datafusion.optimizer.enable_distinct_aggregation_soft_limit | true | When set to true, the optimizer will push a limit operation into grouped aggregations which have no aggregate expressions, as a soft limit, emitting groups once the limit is reached, before all rows in the group are read. | -| datafusion.optimizer.enable_round_robin_repartition | true | When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores | -| datafusion.optimizer.enable_topk_aggregation | true | When set to true, the optimizer will attempt to perform limit operations during aggregations, if possible | -| datafusion.optimizer.enable_window_limits | true | When set to true, the optimizer will attempt to push limit operations past window functions, if possible | -| datafusion.optimizer.enable_topk_dynamic_filter_pushdown | true | When set to true, the optimizer will attempt to push down TopK dynamic filters into the file scan phase. | -| datafusion.optimizer.enable_join_dynamic_filter_pushdown | true | When set to true, the optimizer will attempt to push down Join dynamic filters into the file scan phase. | -| datafusion.optimizer.enable_aggregate_dynamic_filter_pushdown | true | When set to true, the optimizer will attempt to push down Aggregate dynamic filters into the file scan phase. | -| datafusion.optimizer.enable_dynamic_filter_pushdown | true | When set to true attempts to push down dynamic filters generated by operators (TopK, Join & Aggregate) into the file scan phase. For example, for a query such as `SELECT * FROM t ORDER BY timestamp DESC LIMIT 10`, the optimizer will attempt to push down the current top 10 timestamps that the TopK operator references into the file scans. This means that if we already have 10 timestamps in the year 2025 any files that only have timestamps in the year 2024 can be skipped / pruned at various stages in the scan. The config will suppress `enable_join_dynamic_filter_pushdown`, `enable_topk_dynamic_filter_pushdown` & `enable_aggregate_dynamic_filter_pushdown` So if you disable `enable_topk_dynamic_filter_pushdown`, then enable `enable_dynamic_filter_pushdown`, the `enable_topk_dynamic_filter_pushdown` will be overridden. | -| datafusion.optimizer.filter_null_join_keys | false | When set to true, the optimizer will insert filters before a join between a nullable and non-nullable column to filter out nulls on the nullable side. This filter can add additional overhead when the file format does not fully support predicate push down. | -| datafusion.optimizer.repartition_aggregations | true | Should DataFusion repartition data using the aggregate keys to execute aggregates in parallel using the provided `target_partitions` level | -| datafusion.optimizer.repartition_file_min_size | 10485760 | Minimum total files size in bytes to perform file scan repartitioning. | -| datafusion.optimizer.repartition_joins | true | Should DataFusion repartition data using the join keys to execute joins in parallel using the provided `target_partitions` level | -| datafusion.optimizer.allow_symmetric_joins_without_pruning | true | Should DataFusion allow symmetric hash joins for unbounded data sources even when its inputs do not have any ordering or filtering If the flag is not enabled, the SymmetricHashJoin operator will be unable to prune its internal buffers, resulting in certain join types - such as Full, Left, LeftAnti, LeftSemi, Right, RightAnti, and RightSemi - being produced only at the end of the execution. This is not typical in stream processing. Additionally, without proper design for long runner execution, all types of joins may encounter out-of-memory errors. | -| datafusion.optimizer.repartition_file_scans | true | When set to `true`, datasource partitions will be repartitioned to achieve maximum parallelism. This applies to both in-memory partitions and FileSource's file groups (1 group is 1 partition). For FileSources, only Parquet and CSV formats are currently supported. If set to `true` for a FileSource, all files will be repartitioned evenly (i.e., a single large file might be partitioned into smaller chunks) for parallel scanning. If set to `false` for a FileSource, different files will be read in parallel, but repartitioning won't happen within a single file. If set to `true` for an in-memory source, all memtable's partitions will have their batches repartitioned evenly to the desired number of `target_partitions`. Repartitioning can change the total number of partitions and batches per partition, but does not slice the initial record tables provided to the MemTable on creation. | -| datafusion.optimizer.preserve_file_partitions | 0 | Minimum number of distinct partition values required to group files by their Hive partition column values (enabling Hash partitioning declaration). How the option is used: - preserve_file_partitions=0: Disable it. - preserve_file_partitions=1: Always enable it. - preserve_file_partitions=N, actual file partitions=M: Only enable when M >= N. This threshold preserves I/O parallelism when file partitioning is below it. Note: This may reduce parallelism, rooting from the I/O level, if the number of distinct partitions is less than the target_partitions. | -| datafusion.optimizer.repartition_windows | true | Should DataFusion repartition data using the partitions keys to execute window functions in parallel using the provided `target_partitions` level | -| datafusion.optimizer.repartition_sorts | true | Should DataFusion execute sorts in a per-partition fashion and merge afterwards instead of coalescing first and sorting globally. With this flag is enabled, plans in the form below `text "SortExec: [a@0 ASC]", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", ` would turn into the plan below which performs better in multithreaded environments `text "SortPreservingMergeExec: [a@0 ASC]", " SortExec: [a@0 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", ` | -| datafusion.optimizer.subset_repartition_threshold | 4 | Partition count threshold for subset satisfaction optimization. When the current partition count is >= this threshold, DataFusion will skip repartitioning if the required partitioning expression is a subset of the current partition expression such as Hash(a) satisfies Hash(a, b). When the current partition count is < this threshold, DataFusion will repartition to increase parallelism even when subset satisfaction applies. Set to 0 to always repartition (disable subset satisfaction optimization). Set to a high value to always use subset satisfaction. Example (subset_repartition_threshold = 4): `text Hash([a]) satisfies Hash([a, b]) because (Hash([a, b]) is subset of Hash([a]) If current partitions (3) < threshold (4), repartition: AggregateExec: mode=FinalPartitioned, gby=[a, b], aggr=[SUM(x)] RepartitionExec: partitioning=Hash([a, b], 8), input_partitions=3 AggregateExec: mode=Partial, gby=[a, b], aggr=[SUM(x)] DataSourceExec: file_groups={...}, output_partitioning=Hash([a], 3) If current partitions (8) >= threshold (4), use subset satisfaction: AggregateExec: mode=SinglePartitioned, gby=[a, b], aggr=[SUM(x)] DataSourceExec: file_groups={...}, output_partitioning=Hash([a], 8) ` | -| datafusion.optimizer.prefer_existing_sort | false | When true, DataFusion will opportunistically remove sorts when the data is already sorted, (i.e. setting `preserve_order` to true on `RepartitionExec` and using `SortPreservingMergeExec`) When false, DataFusion will maximize plan parallelism using `RepartitionExec` even if this requires subsequently resorting data using a `SortExec`. | -| datafusion.optimizer.skip_failed_rules | false | When set to true, the logical plan optimizer will produce warning messages if any optimization rules produce errors and then proceed to the next rule. When set to false, any rules that produce errors will cause the query to fail | -| datafusion.optimizer.max_passes | 3 | Number of times that the optimizer will attempt to optimize the plan | -| datafusion.optimizer.top_down_join_key_reordering | true | When set to true, the physical plan optimizer will run a top down process to reorder the join keys | -| datafusion.optimizer.prefer_hash_join | true | When set to true, the physical plan optimizer will prefer HashJoin over SortMergeJoin. HashJoin can work more efficiently than SortMergeJoin but consumes more memory | -| datafusion.optimizer.enable_piecewise_merge_join | false | When set to true, piecewise merge join is enabled. PiecewiseMergeJoin is currently experimental. Physical planner will opt for PiecewiseMergeJoin when there is only one range filter. | -| datafusion.optimizer.hash_join_single_partition_threshold | 1048576 | The maximum estimated size in bytes for one input side of a HashJoin will be collected into a single partition | -| datafusion.optimizer.hash_join_single_partition_threshold_rows | 131072 | The maximum estimated size in rows for one input side of a HashJoin will be collected into a single partition | -| datafusion.optimizer.hash_join_inlist_pushdown_max_size | 131072 | Maximum size in bytes for the build side of a hash join to be pushed down as an InList expression for dynamic filtering. Build sides larger than this will use hash table lookups instead. Set to 0 to always use hash table lookups. InList pushdown can be more efficient for small build sides because it can result in better statistics pruning as well as use any bloom filters present on the scan side. InList expressions are also more transparent and easier to serialize over the network in distributed uses of DataFusion. On the other hand InList pushdown requires making a copy of the data and thus adds some overhead to the build side and uses more memory. This setting is per-partition, so we may end up using `hash_join_inlist_pushdown_max_size` \* `target_partitions` memory. The default is 128kB per partition. This should allow point lookup joins (e.g. joining on a unique primary key) to use InList pushdown in most cases but avoids excessive memory usage or overhead for larger joins. | -| datafusion.optimizer.hash_join_inlist_pushdown_max_distinct_values | 150 | Maximum number of distinct values (rows) in the build side of a hash join to be pushed down as an InList expression for dynamic filtering. Build sides with more rows than this will use hash table lookups instead. Set to 0 to always use hash table lookups. This provides an additional limit beyond `hash_join_inlist_pushdown_max_size` to prevent very large IN lists that might not provide much benefit over hash table lookups. This uses the deduplicated row count once the build side has been evaluated. The default is 150 values per partition. This is inspired by Trino's `max-filter-keys-per-column` setting. See: | -| datafusion.optimizer.default_filter_selectivity | 20 | The default filter selectivity used by Filter Statistics when an exact selectivity cannot be determined. Valid values are between 0 (no selectivity) and 100 (all rows are selected). | -| datafusion.optimizer.prefer_existing_union | false | When set to true, the optimizer will not attempt to convert Union to Interleave | -| datafusion.optimizer.expand_views_at_output | false | When set to true, if the returned type is a view type then the output will be coerced to a non-view. Coerces `Utf8View` to `LargeUtf8`, and `BinaryView` to `LargeBinary`. | -| datafusion.optimizer.enable_sort_pushdown | true | Enable sort pushdown optimization. When enabled, attempts to push sort requirements down to data sources that can natively handle them (e.g., by reversing file/row group read order). Returns **inexact ordering**: Sort operator is kept for correctness, but optimized input enables early termination for TopK queries (ORDER BY ... LIMIT N), providing significant speedup. Memory: No additional overhead (only changes read order). Future: Will add option to detect perfectly sorted data and eliminate Sort completely. Default: true | -| datafusion.optimizer.enable_leaf_expression_pushdown | true | When set to true, the optimizer will extract leaf expressions (such as `get_field`) from filter/sort/join nodes into projections closer to the leaf table scans, and push those projections down towards the leaf nodes. | -| datafusion.explain.logical_plan_only | false | When set to true, the explain statement will only print logical plans | -| datafusion.explain.physical_plan_only | false | When set to true, the explain statement will only print physical plans | -| datafusion.explain.show_statistics | false | When set to true, the explain statement will print operator statistics for physical plans | -| datafusion.explain.show_sizes | true | When set to true, the explain statement will print the partition sizes | -| datafusion.explain.show_schema | false | When set to true, the explain statement will print schema information | -| datafusion.explain.format | indent | Display format of explain. Default is "indent". When set to "tree", it will print the plan in a tree-rendered format. | -| datafusion.explain.tree_maximum_render_width | 240 | (format=tree only) Maximum total width of the rendered tree. When set to 0, the tree will have no width limit. | -| datafusion.explain.analyze_level | dev | Verbosity level for "EXPLAIN ANALYZE". Default is "dev" "summary" shows common metrics for high-level insights. "dev" provides deep operator-level introspection for developers. | -| datafusion.sql_parser.parse_float_as_decimal | false | When set to true, SQL parser will parse float as decimal type | -| datafusion.sql_parser.enable_ident_normalization | true | When set to true, SQL parser will normalize ident (convert ident to lowercase when not quoted) | -| datafusion.sql_parser.enable_options_value_normalization | false | When set to true, SQL parser will normalize options value (convert value to lowercase). Note that this option is ignored and will be removed in the future. All case-insensitive values are normalized automatically. | -| datafusion.sql_parser.dialect | generic | Configure the SQL dialect used by DataFusion's parser; supported values include: Generic, MySQL, PostgreSQL, Hive, SQLite, Snowflake, Redshift, MsSQL, ClickHouse, BigQuery, Ansi, DuckDB and Databricks. | -| datafusion.sql_parser.support_varchar_with_length | true | If true, permit lengths for `VARCHAR` such as `VARCHAR(20)`, but ignore the length. If false, error if a `VARCHAR` with a length is specified. The Arrow type system does not have a notion of maximum string length and thus DataFusion can not enforce such limits. | -| datafusion.sql_parser.map_string_types_to_utf8view | true | If true, string types (VARCHAR, CHAR, Text, and String) are mapped to `Utf8View` during SQL planning. If false, they are mapped to `Utf8`. Default is true. | -| datafusion.sql_parser.collect_spans | false | When set to true, the source locations relative to the original SQL query (i.e. [`Span`](https://docs.rs/sqlparser/latest/sqlparser/tokenizer/struct.Span.html)) will be collected and recorded in the logical plan nodes. | -| datafusion.sql_parser.recursion_limit | 50 | Specifies the recursion depth limit when parsing complex SQL Queries | -| datafusion.sql_parser.default_null_ordering | nulls_max | Specifies the default null ordering for query results. There are 4 options: - `nulls_max`: Nulls appear last in ascending order. - `nulls_min`: Nulls appear first in ascending order. - `nulls_first`: Nulls always be first in any order. - `nulls_last`: Nulls always be last in any order. By default, `nulls_max` is used to follow Postgres's behavior. postgres rule: | -| datafusion.format.safe | true | If set to `true` any formatting errors will be written to the output instead of being converted into a [`std::fmt::Error`] | -| datafusion.format.null | | Format string for nulls | -| datafusion.format.date_format | %Y-%m-%d | Date format for date arrays | -| datafusion.format.datetime_format | %Y-%m-%dT%H:%M:%S%.f | Format for DateTime arrays | -| datafusion.format.timestamp_format | %Y-%m-%dT%H:%M:%S%.f | Timestamp format for timestamp arrays | -| datafusion.format.timestamp_tz_format | NULL | Timestamp format for timestamp with timezone arrays. When `None`, ISO 8601 format is used. | -| datafusion.format.time_format | %H:%M:%S%.f | Time format for time arrays | -| datafusion.format.duration_format | pretty | Duration format. Can be either `"pretty"` or `"ISO8601"` | -| datafusion.format.types_info | false | Show types in visual representation batches | +| datafusion.execution.planning_concurrency | 0 | Fan-out during initial physical planning. This is mostly use to plan `UNION` children in parallel. Defaults to the number of CPU cores on the system | +| datafusion.execution.skip_physical_aggregate_schema_check | false | When set to true, skips verifying that the schema produced by planning the input of `LogicalPlan::Aggregate` exactly matches the schema of the input plan. When set to false, if the schema does not match exactly (including nullability and metadata), a planning error will be raised. This is used to workaround bugs in the planner that are now caught by the new schema verification step. | +| datafusion.execution.spill_compression | uncompressed | Sets the compression codec used when spilling data to disk. Since datafusion writes spill files using the Arrow IPC Stream format, only codecs supported by the Arrow IPC Stream Writer are allowed. Valid values are: uncompressed, lz4_frame, zstd. Note: lz4_frame offers faster (de)compression, but typically results in larger spill files. In contrast, zstd achieves higher compression ratios at the cost of slower (de)compression speed. | +| datafusion.execution.sort_spill_reservation_bytes | 10485760 | Specifies the reserved memory for each spillable sort operation to facilitate an in-memory merge. When a sort operation spills to disk, the in-memory data must be sorted and merged before being written to a file. This setting reserves a specific amount of memory for that in-memory sort/merge process. Note: This setting is irrelevant if the sort operation cannot spill (i.e., if there's no `DiskManager` configured). | +| datafusion.execution.sort_in_place_threshold_bytes | 1048576 | When sorting, below what size should data be concatenated and sorted in a single RecordBatch rather than sorted in batches and merged. | +| datafusion.execution.max_spill_file_size_bytes | 134217728 | Maximum size in bytes for individual spill files before rotating to a new file. When operators spill data to disk (e.g., RepartitionExec), they write multiple batches to the same file until this size limit is reached, then rotate to a new file. This reduces syscall overhead compared to one-file-per-batch while preventing files from growing too large. A larger value reduces file creation overhead but may hold more disk space. A smaller value creates more files but allows finer-grained space reclamation as files can be deleted once fully consumed. Now only `RepartitionExec` supports this spill file rotation feature, other spilling operators may create spill files larger than the limit. Default: 128 MB | +| datafusion.execution.meta_fetch_concurrency | 32 | Number of files to read in parallel when inferring schema and statistics | +| datafusion.execution.minimum_parallel_output_files | 4 | Guarantees a minimum level of output files running in parallel. RecordBatches will be distributed in round robin fashion to each parallel writer. Each writer is closed and a new file opened once soft_max_rows_per_output_file is reached. | +| datafusion.execution.soft_max_rows_per_output_file | 50000000 | Target number of rows in output files when writing multiple. This is a soft max, so it can be exceeded slightly. There also will be one file smaller than the limit if the total number of rows written is not roughly divisible by the soft max | +| datafusion.execution.max_buffered_batches_per_output_file | 2 | This is the maximum number of RecordBatches buffered for each output file being worked. Higher values can potentially give faster write performance at the cost of higher peak memory consumption | +| datafusion.execution.listing_table_ignore_subdirectory | true | Should sub directories be ignored when scanning directories for data files. Defaults to true (ignores subdirectories), consistent with Hive. Note that this setting does not affect reading partitioned tables (e.g. `/table/year=2021/month=01/data.parquet`). | +| datafusion.execution.listing_table_factory_infer_partitions | true | Should a `ListingTable` created through the `ListingTableFactory` infer table partitions from Hive compliant directories. Defaults to true (partition columns are inferred and will be represented in the table schema). | +| datafusion.execution.enable_recursive_ctes | true | Should DataFusion support recursive CTEs | +| datafusion.execution.split_file_groups_by_statistics | false | Attempt to eliminate sorts by packing & sorting files with non-overlapping statistics into the same file groups. Currently experimental | +| datafusion.execution.keep_partition_by_columns | false | Should DataFusion keep the columns used for partition_by in the output RecordBatches | +| datafusion.execution.skip_partial_aggregation_probe_ratio_threshold | 0.8 | Aggregation ratio (number of distinct groups / number of input rows) threshold for skipping partial aggregation. If the value is greater then partial aggregation will skip aggregation for further input | +| datafusion.execution.skip_partial_aggregation_probe_rows_threshold | 100000 | Number of input rows partial aggregation partition should process, before aggregation ratio check and trying to switch to skipping aggregation mode | +| datafusion.execution.use_row_number_estimates_to_optimize_partitioning | false | Should DataFusion use row number estimates at the input to decide whether increasing parallelism is beneficial or not. By default, only exact row numbers (not estimates) are used for this decision. Setting this flag to `true` will likely produce better plans. if the source of statistics is accurate. We plan to make this the default in the future. | +| datafusion.execution.enforce_batch_size_in_joins | false | Should DataFusion enforce batch size in joins or not. By default, DataFusion will not enforce batch size in joins. Enforcing batch size in joins can reduce memory usage when joining large tables with a highly-selective join filter, but is also slightly slower. | +| datafusion.execution.objectstore_writer_buffer_size | 10485760 | Size (bytes) of data buffer DataFusion uses when writing output files. This affects the size of the data chunks that are uploaded to remote object stores (e.g. AWS S3). If very large (>= 100 GiB) output files are being written, it may be necessary to increase this size to avoid errors from the remote end point. | +| datafusion.execution.enable_ansi_mode | false | Whether to enable ANSI SQL mode. The flag is experimental and relevant only for DataFusion Spark built-in functions When `enable_ansi_mode` is set to `true`, the query engine follows ANSI SQL semantics for expressions, casting, and error handling. This means: - **Strict type coercion rules:** implicit casts between incompatible types are disallowed. - **Standard SQL arithmetic behavior:** operations such as division by zero, numeric overflow, or invalid casts raise runtime errors rather than returning `NULL` or adjusted values. - **Consistent ANSI behavior** for string concatenation, comparisons, and `NULL` handling. When `enable_ansi_mode` is `false` (the default), the engine uses a more permissive, non-ANSI mode designed for user convenience and backward compatibility. In this mode: - Implicit casts between types are allowed (e.g., string to integer when possible). - Arithmetic operations are more lenient — for example, `abs()` on the minimum representable integer value returns the input value instead of raising overflow. - Division by zero or invalid casts may return `NULL` instead of failing. # Default `false` — ANSI SQL mode is disabled by default. | +| datafusion.optimizer.enable_distinct_aggregation_soft_limit | true | When set to true, the optimizer will push a limit operation into grouped aggregations which have no aggregate expressions, as a soft limit, emitting groups once the limit is reached, before all rows in the group are read. | +| datafusion.optimizer.enable_round_robin_repartition | true | When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores | +| datafusion.optimizer.enable_topk_aggregation | true | When set to true, the optimizer will attempt to perform limit operations during aggregations, if possible | +| datafusion.optimizer.enable_window_limits | true | When set to true, the optimizer will attempt to push limit operations past window functions, if possible | +| datafusion.optimizer.enable_topk_dynamic_filter_pushdown | true | When set to true, the optimizer will attempt to push down TopK dynamic filters into the file scan phase. | +| datafusion.optimizer.enable_join_dynamic_filter_pushdown | true | When set to true, the optimizer will attempt to push down Join dynamic filters into the file scan phase. | +| datafusion.optimizer.enable_aggregate_dynamic_filter_pushdown | true | When set to true, the optimizer will attempt to push down Aggregate dynamic filters into the file scan phase. | +| datafusion.optimizer.enable_dynamic_filter_pushdown | true | When set to true attempts to push down dynamic filters generated by operators (TopK, Join & Aggregate) into the file scan phase. For example, for a query such as `SELECT * FROM t ORDER BY timestamp DESC LIMIT 10`, the optimizer will attempt to push down the current top 10 timestamps that the TopK operator references into the file scans. This means that if we already have 10 timestamps in the year 2025 any files that only have timestamps in the year 2024 can be skipped / pruned at various stages in the scan. The config will suppress `enable_join_dynamic_filter_pushdown`, `enable_topk_dynamic_filter_pushdown` & `enable_aggregate_dynamic_filter_pushdown` So if you disable `enable_topk_dynamic_filter_pushdown`, then enable `enable_dynamic_filter_pushdown`, the `enable_topk_dynamic_filter_pushdown` will be overridden. | +| datafusion.optimizer.filter_null_join_keys | false | When set to true, the optimizer will insert filters before a join between a nullable and non-nullable column to filter out nulls on the nullable side. This filter can add additional overhead when the file format does not fully support predicate push down. | +| datafusion.optimizer.repartition_aggregations | true | Should DataFusion repartition data using the aggregate keys to execute aggregates in parallel using the provided `target_partitions` level | +| datafusion.optimizer.repartition_file_min_size | 10485760 | Minimum total files size in bytes to perform file scan repartitioning. | +| datafusion.optimizer.repartition_joins | true | Should DataFusion repartition data using the join keys to execute joins in parallel using the provided `target_partitions` level | +| datafusion.optimizer.allow_symmetric_joins_without_pruning | true | Should DataFusion allow symmetric hash joins for unbounded data sources even when its inputs do not have any ordering or filtering If the flag is not enabled, the SymmetricHashJoin operator will be unable to prune its internal buffers, resulting in certain join types - such as Full, Left, LeftAnti, LeftSemi, Right, RightAnti, and RightSemi - being produced only at the end of the execution. This is not typical in stream processing. Additionally, without proper design for long runner execution, all types of joins may encounter out-of-memory errors. | +| datafusion.optimizer.repartition_file_scans | true | When set to `true`, datasource partitions will be repartitioned to achieve maximum parallelism. This applies to both in-memory partitions and FileSource's file groups (1 group is 1 partition). For FileSources, only Parquet and CSV formats are currently supported. If set to `true` for a FileSource, all files will be repartitioned evenly (i.e., a single large file might be partitioned into smaller chunks) for parallel scanning. If set to `false` for a FileSource, different files will be read in parallel, but repartitioning won't happen within a single file. If set to `true` for an in-memory source, all memtable's partitions will have their batches repartitioned evenly to the desired number of `target_partitions`. Repartitioning can change the total number of partitions and batches per partition, but does not slice the initial record tables provided to the MemTable on creation. | +| datafusion.optimizer.preserve_file_partitions | 0 | Minimum number of distinct partition values required to group files by their Hive partition column values (enabling Hash partitioning declaration). How the option is used: - preserve_file_partitions=0: Disable it. - preserve_file_partitions=1: Always enable it. - preserve_file_partitions=N, actual file partitions=M: Only enable when M >= N. This threshold preserves I/O parallelism when file partitioning is below it. Note: This may reduce parallelism, rooting from the I/O level, if the number of distinct partitions is less than the target_partitions. | +| datafusion.optimizer.repartition_windows | true | Should DataFusion repartition data using the partitions keys to execute window functions in parallel using the provided `target_partitions` level | +| datafusion.optimizer.repartition_sorts | true | Should DataFusion execute sorts in a per-partition fashion and merge afterwards instead of coalescing first and sorting globally. With this flag is enabled, plans in the form below `text "SortExec: [a@0 ASC]", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", ` would turn into the plan below which performs better in multithreaded environments `text "SortPreservingMergeExec: [a@0 ASC]", " SortExec: [a@0 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", ` | +| datafusion.optimizer.subset_repartition_threshold | 4 | Partition count threshold for subset satisfaction optimization. When the current partition count is >= this threshold, DataFusion will skip repartitioning if the required partitioning expression is a subset of the current partition expression such as Hash(a) satisfies Hash(a, b). When the current partition count is < this threshold, DataFusion will repartition to increase parallelism even when subset satisfaction applies. Set to 0 to always repartition (disable subset satisfaction optimization). Set to a high value to always use subset satisfaction. Example (subset_repartition_threshold = 4): `text Hash([a]) satisfies Hash([a, b]) because (Hash([a, b]) is subset of Hash([a]) If current partitions (3) < threshold (4), repartition: AggregateExec: mode=FinalPartitioned, gby=[a, b], aggr=[SUM(x)] RepartitionExec: partitioning=Hash([a, b], 8), input_partitions=3 AggregateExec: mode=Partial, gby=[a, b], aggr=[SUM(x)] DataSourceExec: file_groups={...}, output_partitioning=Hash([a], 3) If current partitions (8) >= threshold (4), use subset satisfaction: AggregateExec: mode=SinglePartitioned, gby=[a, b], aggr=[SUM(x)] DataSourceExec: file_groups={...}, output_partitioning=Hash([a], 8) ` | +| datafusion.optimizer.prefer_existing_sort | false | When true, DataFusion will opportunistically remove sorts when the data is already sorted, (i.e. setting `preserve_order` to true on `RepartitionExec` and using `SortPreservingMergeExec`) When false, DataFusion will maximize plan parallelism using `RepartitionExec` even if this requires subsequently resorting data using a `SortExec`. | +| datafusion.optimizer.skip_failed_rules | false | When set to true, the logical plan optimizer will produce warning messages if any optimization rules produce errors and then proceed to the next rule. When set to false, any rules that produce errors will cause the query to fail | +| datafusion.optimizer.max_passes | 3 | Number of times that the optimizer will attempt to optimize the plan | +| datafusion.optimizer.top_down_join_key_reordering | true | When set to true, the physical plan optimizer will run a top down process to reorder the join keys | +| datafusion.optimizer.prefer_hash_join | true | When set to true, the physical plan optimizer will prefer HashJoin over SortMergeJoin. HashJoin can work more efficiently than SortMergeJoin but consumes more memory | +| datafusion.optimizer.enable_piecewise_merge_join | false | When set to true, piecewise merge join is enabled. PiecewiseMergeJoin is currently experimental. Physical planner will opt for PiecewiseMergeJoin when there is only one range filter. | +| datafusion.optimizer.hash_join_single_partition_threshold | 1048576 | The maximum estimated size in bytes for one input side of a HashJoin will be collected into a single partition | +| datafusion.optimizer.hash_join_single_partition_threshold_rows | 131072 | The maximum estimated size in rows for one input side of a HashJoin will be collected into a single partition | +| datafusion.optimizer.hash_join_inlist_pushdown_max_size | 131072 | Maximum size in bytes for the build side of a hash join to be pushed down as an InList expression for dynamic filtering. Build sides larger than this will use hash table lookups instead. Set to 0 to always use hash table lookups. InList pushdown can be more efficient for small build sides because it can result in better statistics pruning as well as use any bloom filters present on the scan side. InList expressions are also more transparent and easier to serialize over the network in distributed uses of DataFusion. On the other hand InList pushdown requires making a copy of the data and thus adds some overhead to the build side and uses more memory. This setting is per-partition, so we may end up using `hash_join_inlist_pushdown_max_size` \* `target_partitions` memory. The default is 128kB per partition. This should allow point lookup joins (e.g. joining on a unique primary key) to use InList pushdown in most cases but avoids excessive memory usage or overhead for larger joins. | +| datafusion.optimizer.hash_join_inlist_pushdown_max_distinct_values | 150 | Maximum number of distinct values (rows) in the build side of a hash join to be pushed down as an InList expression for dynamic filtering. Build sides with more rows than this will use hash table lookups instead. Set to 0 to always use hash table lookups. This provides an additional limit beyond `hash_join_inlist_pushdown_max_size` to prevent very large IN lists that might not provide much benefit over hash table lookups. This uses the deduplicated row count once the build side has been evaluated. The default is 150 values per partition. This is inspired by Trino's `max-filter-keys-per-column` setting. See: | +| datafusion.optimizer.default_filter_selectivity | 20 | The default filter selectivity used by Filter Statistics when an exact selectivity cannot be determined. Valid values are between 0 (no selectivity) and 100 (all rows are selected). | +| datafusion.optimizer.prefer_existing_union | false | When set to true, the optimizer will not attempt to convert Union to Interleave | +| datafusion.optimizer.expand_views_at_output | false | When set to true, if the returned type is a view type then the output will be coerced to a non-view. Coerces `Utf8View` to `LargeUtf8`, and `BinaryView` to `LargeBinary`. | +| datafusion.optimizer.enable_sort_pushdown | true | Enable sort pushdown optimization. When enabled, attempts to push sort requirements down to data sources that can natively handle them (e.g., by reversing file/row group read order). Returns **inexact ordering**: Sort operator is kept for correctness, but optimized input enables early termination for TopK queries (ORDER BY ... LIMIT N), providing significant speedup. Memory: No additional overhead (only changes read order). Future: Will add option to detect perfectly sorted data and eliminate Sort completely. Default: true | +| datafusion.optimizer.enable_leaf_expression_pushdown | true | When set to true, the optimizer will extract leaf expressions (such as `get_field`) from filter/sort/join nodes into projections closer to the leaf table scans, and push those projections down towards the leaf nodes. | +| datafusion.explain.logical_plan_only | false | When set to true, the explain statement will only print logical plans | +| datafusion.explain.physical_plan_only | false | When set to true, the explain statement will only print physical plans | +| datafusion.explain.show_statistics | false | When set to true, the explain statement will print operator statistics for physical plans | +| datafusion.explain.show_sizes | true | When set to true, the explain statement will print the partition sizes | +| datafusion.explain.show_schema | false | When set to true, the explain statement will print schema information | +| datafusion.explain.format | indent | Display format of explain. Default is "indent". When set to "tree", it will print the plan in a tree-rendered format. | +| datafusion.explain.tree_maximum_render_width | 240 | (format=tree only) Maximum total width of the rendered tree. When set to 0, the tree will have no width limit. | +| datafusion.explain.analyze_level | dev | Verbosity level for "EXPLAIN ANALYZE". Default is "dev" "summary" shows common metrics for high-level insights. "dev" provides deep operator-level introspection for developers. | +| datafusion.sql_parser.parse_float_as_decimal | false | When set to true, SQL parser will parse float as decimal type | +| datafusion.sql_parser.enable_ident_normalization | true | When set to true, SQL parser will normalize ident (convert ident to lowercase when not quoted) | +| datafusion.sql_parser.enable_options_value_normalization | false | When set to true, SQL parser will normalize options value (convert value to lowercase). Note that this option is ignored and will be removed in the future. All case-insensitive values are normalized automatically. | +| datafusion.sql_parser.dialect | generic | Configure the SQL dialect used by DataFusion's parser; supported values include: Generic, MySQL, PostgreSQL, Hive, SQLite, Snowflake, Redshift, MsSQL, ClickHouse, BigQuery, Ansi, DuckDB and Databricks. | +| datafusion.sql_parser.support_varchar_with_length | true | If true, permit lengths for `VARCHAR` such as `VARCHAR(20)`, but ignore the length. If false, error if a `VARCHAR` with a length is specified. The Arrow type system does not have a notion of maximum string length and thus DataFusion can not enforce such limits. | +| datafusion.sql_parser.map_string_types_to_utf8view | true | If true, string types (VARCHAR, CHAR, Text, and String) are mapped to `Utf8View` during SQL planning. If false, they are mapped to `Utf8`. Default is true. | +| datafusion.sql_parser.collect_spans | false | When set to true, the source locations relative to the original SQL query (i.e. [`Span`](https://docs.rs/sqlparser/latest/sqlparser/tokenizer/struct.Span.html)) will be collected and recorded in the logical plan nodes. | +| datafusion.sql_parser.recursion_limit | 50 | Specifies the recursion depth limit when parsing complex SQL Queries | +| datafusion.sql_parser.default_null_ordering | nulls_max | Specifies the default null ordering for query results. There are 4 options: - `nulls_max`: Nulls appear last in ascending order. - `nulls_min`: Nulls appear first in ascending order. - `nulls_first`: Nulls always be first in any order. - `nulls_last`: Nulls always be last in any order. By default, `nulls_max` is used to follow Postgres's behavior. postgres rule: | +| datafusion.format.safe | true | If set to `true` any formatting errors will be written to the output instead of being converted into a [`std::fmt::Error`] | +| datafusion.format.null | | Format string for nulls | +| datafusion.format.date_format | %Y-%m-%d | Date format for date arrays | +| datafusion.format.datetime_format | %Y-%m-%dT%H:%M:%S%.f | Format for DateTime arrays | +| datafusion.format.timestamp_format | %Y-%m-%dT%H:%M:%S%.f | Timestamp format for timestamp arrays | +| datafusion.format.timestamp_tz_format | NULL | Timestamp format for timestamp with timezone arrays. When `None`, ISO 8601 format is used. | +| datafusion.format.time_format | %H:%M:%S%.f | Time format for time arrays | +| datafusion.format.duration_format | pretty | Duration format. Can be either `"pretty"` or `"ISO8601"` | +| datafusion.format.types_info | false | Show types in visual representation batches | # Runtime Configuration Settings From 41d3d7866040fa9a1fdd06b3203486dc93b53a0a Mon Sep 17 00:00:00 2001 From: Subham Singhal Date: Sun, 22 Feb 2026 22:22:36 +0530 Subject: [PATCH 08/15] Fix UT --- datafusion/common/src/config.rs | 2 +- datafusion/datasource-parquet/src/file_format.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index ec520577d0929..7a207920d7549 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -751,7 +751,7 @@ config_namespace! { /// parquet reader setting. 0 means no caching. pub max_predicate_cache_size: Option, default = None - /// If true, use Parquet field IDs for column resolution instead of + /// (reading) If true, use Parquet field IDs for column resolution instead of /// column names. This enables schema evolution with renamed/reordered columns. /// When field IDs are unavailable, falls back to name-based matching. pub field_id_enabled: bool, default = false diff --git a/datafusion/datasource-parquet/src/file_format.rs b/datafusion/datasource-parquet/src/file_format.rs index e0fbe356db3b4..d65d331f89203 100644 --- a/datafusion/datasource-parquet/src/file_format.rs +++ b/datafusion/datasource-parquet/src/file_format.rs @@ -283,7 +283,7 @@ impl ParquetFormat { self } - /// Get whether field ID reading is enabled from options + /// Get whether field ID is enabled from options pub fn field_id_enabled(&self) -> bool { self.options.global.field_id_enabled } From 4d5f47f11dadaddd9a0bed48465e98b64f66ccc3 Mon Sep 17 00:00:00 2001 From: Subham Singhal Date: Mon, 23 Feb 2026 10:39:48 +0530 Subject: [PATCH 09/15] Remove is_nested function --- datafusion/core/tests/parquet/field_id.rs | 29 ++++++++++++++++++- .../datasource-parquet/src/file_format.rs | 2 ++ datafusion/datasource-parquet/src/metadata.rs | 21 +------------- 3 files changed, 31 insertions(+), 21 deletions(-) diff --git a/datafusion/core/tests/parquet/field_id.rs b/datafusion/core/tests/parquet/field_id.rs index a6faa175204ce..62497b7155338 100644 --- a/datafusion/core/tests/parquet/field_id.rs +++ b/datafusion/core/tests/parquet/field_id.rs @@ -48,7 +48,34 @@ fn create_parquet_file_with_field_ids( Ok(()) } -/// Helper to create a schema with field IDs in metadata +/// Creates an Arrow schema with field IDs stored in metadata +/// +/// # Arguments +/// * `fields` - Vector of `(name, data_type, field_id)` tuples +/// +/// # Returns +/// Arrow `Schema` (in-memory) with field IDs in field metadata. +/// When written to Parquet, field IDs are transferred to the Parquet schema. +/// +/// # Example +/// ``` +/// schema_with_field_ids(vec![ +/// ("id".to_string(), DataType::Int32, 1), +/// ("name".to_string(), DataType::Utf8, 2), +/// ]); +/// ``` +/// +/// Creates Arrow schema (in-memory): +/// ``` +/// Field("id", Int32, metadata={"PARQUET:field_id": "1"}) +/// Field("name", Utf8, metadata={"PARQUET:field_id": "2"}) +/// ``` +/// +/// When written to Parquet, produces Parquet schema (on-disk): +/// ``` +/// Column[0]: "id" (INT32, field_id=1) +/// Column[1]: "name" (BYTE_ARRAY/UTF8, field_id=2) +/// ``` fn schema_with_field_ids(fields: Vec<(String, DataType, i32)>) -> Schema { use datafusion_common::parquet_config::PARQUET_FIELD_ID_META_KEY; diff --git a/datafusion/datasource-parquet/src/file_format.rs b/datafusion/datasource-parquet/src/file_format.rs index d65d331f89203..5c1f57cfe52e9 100644 --- a/datafusion/datasource-parquet/src/file_format.rs +++ b/datafusion/datasource-parquet/src/file_format.rs @@ -633,6 +633,8 @@ impl ParquetFormat { /// # Arguments /// * `table_schema` - The table schema containing the desired types /// * `file_schema` - The file schema to be transformed +/// * `enable_field_ids` - If true, matches columns by field ID first, then falls back to name. +/// If false, matches columns by name only /// /// # Returns /// * `Some(Schema)` - If any transformations were applied, returns the transformed schema diff --git a/datafusion/datasource-parquet/src/metadata.rs b/datafusion/datasource-parquet/src/metadata.rs index 41a3c0372253e..269fc4f93b7a4 100644 --- a/datafusion/datasource-parquet/src/metadata.rs +++ b/datafusion/datasource-parquet/src/metadata.rs @@ -95,7 +95,7 @@ fn add_field_ids_to_arrow_schema( // Validate that schema is flat (no nested types) // This prevents incorrect field ID assignment for complex types for (idx, field) in arrow_schema.fields().iter().enumerate() { - if is_nested_type(field.data_type()) { + if field.data_type().is_nested() { return not_impl_err!( "Field ID reading is not yet supported for nested/complex types. \ Field '{}' at index {} has type {:?}.", @@ -136,23 +136,6 @@ fn add_field_ids_to_arrow_schema( )) } -/// Helper function to check if a data type is nested/complex -fn is_nested_type(data_type: &DataType) -> bool { - matches!( - data_type, - DataType::List(_) - | DataType::LargeList(_) - | DataType::FixedSizeList(_, _) - | DataType::Struct(_) - | DataType::Union(_, _) - | DataType::Map(_, _) - | DataType::Dictionary(_, _) - | DataType::RunEndEncoded(_, _) - | DataType::ListView(_) - | DataType::LargeListView(_) - ) -} - impl<'a> DFParquetMetadata<'a> { pub fn new(store: &'a dyn ObjectStore, object_meta: &'a ObjectMeta) -> Self { Self { @@ -282,7 +265,6 @@ impl<'a> DFParquetMetadata<'a> { add_field_ids_to_arrow_schema(&schema, file_metadata.schema_descr())?; } - // Apply INT96 coercion if configured let schema = self .coerce_int96 .as_ref() @@ -294,7 +276,6 @@ impl<'a> DFParquetMetadata<'a> { ) }) .unwrap_or(schema); - Ok(schema) } From 288c10babdb8c39828baddf07219828e814b6dd4 Mon Sep 17 00:00:00 2001 From: Subham Singhal Date: Sat, 28 Feb 2026 00:04:57 +0530 Subject: [PATCH 10/15] Resolve comments --- datafusion/datasource-parquet/src/metadata.rs | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/datafusion/datasource-parquet/src/metadata.rs b/datafusion/datasource-parquet/src/metadata.rs index 269fc4f93b7a4..50fc577e246d2 100644 --- a/datafusion/datasource-parquet/src/metadata.rs +++ b/datafusion/datasource-parquet/src/metadata.rs @@ -115,11 +115,11 @@ fn add_field_ids_to_arrow_schema( let col_desc = parquet_schema.column(idx); // Extract field ID from the schema type - // Field IDs are optional in Parquet; if not set, they may be 0 or negative - let field_id = col_desc.self_type().get_basic_info().id(); - - if field_id > 0 { - // Add field ID to field metadata + // Field IDs are optional in Parquet. Valid field IDs are positive integers(> 0). + // if not set, skip adding to metadata + let basic_info = col_desc.self_type().get_basic_info(); + if basic_info.has_id() && basic_info.id() > 0 { + let field_id = basic_info.id(); let mut metadata = field.metadata().clone(); metadata .insert(PARQUET_FIELD_ID_META_KEY.to_string(), field_id.to_string()); From aa2f1655c1287ba44132e3fa377d473134d3b334 Mon Sep 17 00:00:00 2001 From: Subham Singhal Date: Sat, 28 Feb 2026 23:00:10 +0530 Subject: [PATCH 11/15] Fix proto serialisation --- .../proto-common/proto/datafusion_common.proto | 1 + datafusion/proto-common/src/from_proto/mod.rs | 2 +- .../proto-common/src/generated/pbjson.rs | 18 ++++++++++++++++++ datafusion/proto-common/src/generated/prost.rs | 3 +++ datafusion/proto-common/src/to_proto/mod.rs | 1 + 5 files changed, 24 insertions(+), 1 deletion(-) diff --git a/datafusion/proto-common/proto/datafusion_common.proto b/datafusion/proto-common/proto/datafusion_common.proto index 62c6bbe85612a..26dec5aeb4e81 100644 --- a/datafusion/proto-common/proto/datafusion_common.proto +++ b/datafusion/proto-common/proto/datafusion_common.proto @@ -549,6 +549,7 @@ message ParquetOptions { bool schema_force_view_types = 28; // default = false bool binary_as_string = 29; // default = false bool skip_arrow_metadata = 30; // default = false + bool field_id_enabled = 35; // default = false oneof metadata_size_hint_opt { uint64 metadata_size_hint = 4; diff --git a/datafusion/proto-common/src/from_proto/mod.rs b/datafusion/proto-common/src/from_proto/mod.rs index 487d6c40565da..322f51688d165 100644 --- a/datafusion/proto-common/src/from_proto/mod.rs +++ b/datafusion/proto-common/src/from_proto/mod.rs @@ -1090,7 +1090,7 @@ impl TryFrom<&protobuf::ParquetOptions> for ParquetOptions { max_predicate_cache_size: value.max_predicate_cache_size_opt.map(|opt| match opt { protobuf::parquet_options::MaxPredicateCacheSizeOpt::MaxPredicateCacheSize(v) => Some(v as usize), }).unwrap_or(None), - field_id_enabled: false, // Default value + field_id_enabled: value.field_id_enabled, }) } } diff --git a/datafusion/proto-common/src/generated/pbjson.rs b/datafusion/proto-common/src/generated/pbjson.rs index b00e7546bba20..222110c5afe22 100644 --- a/datafusion/proto-common/src/generated/pbjson.rs +++ b/datafusion/proto-common/src/generated/pbjson.rs @@ -5683,6 +5683,9 @@ impl serde::Serialize for ParquetOptions { if self.skip_arrow_metadata { len += 1; } + if self.field_id_enabled { + len += 1; + } if self.dictionary_page_size_limit != 0 { len += 1; } @@ -5788,6 +5791,9 @@ impl serde::Serialize for ParquetOptions { if self.skip_arrow_metadata { struct_ser.serialize_field("skipArrowMetadata", &self.skip_arrow_metadata)?; } + if self.field_id_enabled { + struct_ser.serialize_field("fieldIdEnabled", &self.field_id_enabled)?; + } if self.dictionary_page_size_limit != 0 { #[allow(clippy::needless_borrow)] #[allow(clippy::needless_borrows_for_generic_args)] @@ -5936,6 +5942,8 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { "binaryAsString", "skip_arrow_metadata", "skipArrowMetadata", + "field_id_enabled", + "fieldIdEnabled", "dictionary_page_size_limit", "dictionaryPageSizeLimit", "data_page_row_count_limit", @@ -5985,6 +5993,7 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { SchemaForceViewTypes, BinaryAsString, SkipArrowMetadata, + FieldIdEnabled, DictionaryPageSizeLimit, DataPageRowCountLimit, MaxRowGroupSize, @@ -6038,6 +6047,7 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { "schemaForceViewTypes" | "schema_force_view_types" => Ok(GeneratedField::SchemaForceViewTypes), "binaryAsString" | "binary_as_string" => Ok(GeneratedField::BinaryAsString), "skipArrowMetadata" | "skip_arrow_metadata" => Ok(GeneratedField::SkipArrowMetadata), + "fieldIdEnabled" | "field_id_enabled" => Ok(GeneratedField::FieldIdEnabled), "dictionaryPageSizeLimit" | "dictionary_page_size_limit" => Ok(GeneratedField::DictionaryPageSizeLimit), "dataPageRowCountLimit" | "data_page_row_count_limit" => Ok(GeneratedField::DataPageRowCountLimit), "maxRowGroupSize" | "max_row_group_size" => Ok(GeneratedField::MaxRowGroupSize), @@ -6089,6 +6099,7 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { let mut schema_force_view_types__ = None; let mut binary_as_string__ = None; let mut skip_arrow_metadata__ = None; + let mut field_id_enabled__ = None; let mut dictionary_page_size_limit__ = None; let mut data_page_row_count_limit__ = None; let mut max_row_group_size__ = None; @@ -6216,6 +6227,12 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { } skip_arrow_metadata__ = Some(map_.next_value()?); } + GeneratedField::FieldIdEnabled => { + if field_id_enabled__.is_some() { + return Err(serde::de::Error::duplicate_field("fieldIdEnabled")); + } + field_id_enabled__ = Some(map_.next_value()?); + } GeneratedField::DictionaryPageSizeLimit => { if dictionary_page_size_limit__.is_some() { return Err(serde::de::Error::duplicate_field("dictionaryPageSizeLimit")); @@ -6332,6 +6349,7 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { schema_force_view_types: schema_force_view_types__.unwrap_or_default(), binary_as_string: binary_as_string__.unwrap_or_default(), skip_arrow_metadata: skip_arrow_metadata__.unwrap_or_default(), + field_id_enabled: field_id_enabled__.unwrap_or_default(), dictionary_page_size_limit: dictionary_page_size_limit__.unwrap_or_default(), data_page_row_count_limit: data_page_row_count_limit__.unwrap_or_default(), max_row_group_size: max_row_group_size__.unwrap_or_default(), diff --git a/datafusion/proto-common/src/generated/prost.rs b/datafusion/proto-common/src/generated/prost.rs index a09826a29be52..3a4617bd33644 100644 --- a/datafusion/proto-common/src/generated/prost.rs +++ b/datafusion/proto-common/src/generated/prost.rs @@ -830,6 +830,9 @@ pub struct ParquetOptions { /// default = false #[prost(bool, tag = "30")] pub skip_arrow_metadata: bool, + /// default = false + #[prost(bool, tag = "35")] + pub field_id_enabled: bool, #[prost(uint64, tag = "12")] pub dictionary_page_size_limit: u64, #[prost(uint64, tag = "18")] diff --git a/datafusion/proto-common/src/to_proto/mod.rs b/datafusion/proto-common/src/to_proto/mod.rs index 79e3306a4df1b..844f2c51c499e 100644 --- a/datafusion/proto-common/src/to_proto/mod.rs +++ b/datafusion/proto-common/src/to_proto/mod.rs @@ -904,6 +904,7 @@ impl TryFrom<&ParquetOptions> for protobuf::ParquetOptions { skip_arrow_metadata: value.skip_arrow_metadata, coerce_int96_opt: value.coerce_int96.clone().map(protobuf::parquet_options::CoerceInt96Opt::CoerceInt96), max_predicate_cache_size_opt: value.max_predicate_cache_size.map(|v| protobuf::parquet_options::MaxPredicateCacheSizeOpt::MaxPredicateCacheSize(v as u64)), + field_id_enabled: value.field_id_enabled, }) } } From 444f36ec6b994be59ff2e3e10654ff29b7c3ee82 Mon Sep 17 00:00:00 2001 From: Subham Singhal Date: Sat, 28 Feb 2026 23:20:01 +0530 Subject: [PATCH 12/15] Remove additional whitespace --- datafusion/common/src/config.rs | 32 ++++++++++++++++---------------- 1 file changed, 16 insertions(+), 16 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 7a207920d7549..875b006021481 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -3076,22 +3076,22 @@ config_namespace! { /// If not specified, the default level for the compression algorithm is used. pub compression_level: Option, default = None pub schema_infer_max_rec: Option, default = None - /// The JSON format to use when reading files. - /// - /// When `true` (default), expects newline-delimited JSON (NDJSON): - /// ```text - /// {"key1": 1, "key2": "val"} - /// {"key1": 2, "key2": "vals"} - /// ``` - /// - /// When `false`, expects JSON array format: - /// ```text - /// [ - /// {"key1": 1, "key2": "val"}, - /// {"key1": 2, "key2": "vals"} - /// ] - /// ``` - pub newline_delimited: bool, default = true + /// The JSON format to use when reading files. + /// + /// When `true` (default), expects newline-delimited JSON (NDJSON): + /// ```text + /// {"key1": 1, "key2": "val"} + /// {"key1": 2, "key2": "vals"} + /// ``` + /// + /// When `false`, expects JSON array format: + /// ```text + /// [ + /// {"key1": 1, "key2": "val"}, + /// {"key1": 2, "key2": "vals"} + /// ] + /// ``` + pub newline_delimited: bool, default = true } } From 35419331b05bcc5bc9fd84b2230b1dc4155517b5 Mon Sep 17 00:00:00 2001 From: Subham Singhal Date: Sun, 1 Mar 2026 08:33:52 +0530 Subject: [PATCH 13/15] Resolve build failure --- datafusion/proto/src/generated/datafusion_proto_common.rs | 3 +++ 1 file changed, 3 insertions(+) diff --git a/datafusion/proto/src/generated/datafusion_proto_common.rs b/datafusion/proto/src/generated/datafusion_proto_common.rs index a09826a29be52..3a4617bd33644 100644 --- a/datafusion/proto/src/generated/datafusion_proto_common.rs +++ b/datafusion/proto/src/generated/datafusion_proto_common.rs @@ -830,6 +830,9 @@ pub struct ParquetOptions { /// default = false #[prost(bool, tag = "30")] pub skip_arrow_metadata: bool, + /// default = false + #[prost(bool, tag = "35")] + pub field_id_enabled: bool, #[prost(uint64, tag = "12")] pub dictionary_page_size_limit: u64, #[prost(uint64, tag = "18")] From 1aba3916ca2fa6cd10c93d4b34aa5f29d9f39241 Mon Sep 17 00:00:00 2001 From: Subham Singhal Date: Sun, 1 Mar 2026 11:14:44 +0530 Subject: [PATCH 14/15] Fix build failure --- datafusion/proto/src/logical_plan/file_formats.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/datafusion/proto/src/logical_plan/file_formats.rs b/datafusion/proto/src/logical_plan/file_formats.rs index cf7142bc7cfde..4b5749be157cd 100644 --- a/datafusion/proto/src/logical_plan/file_formats.rs +++ b/datafusion/proto/src/logical_plan/file_formats.rs @@ -426,6 +426,7 @@ mod parquet { max_predicate_cache_size_opt: global_options.global.max_predicate_cache_size.map(|size| { parquet_options::MaxPredicateCacheSizeOpt::MaxPredicateCacheSize(size as u64) }), + field_id_enabled: global_options.global.field_id_enabled, }), column_specific_options: column_specific_options.into_iter().map(|(column_name, options)| { ParquetColumnSpecificOptions { From 385477a710b06b718c5f23364f0a1edc7a4879d9 Mon Sep 17 00:00:00 2001 From: Subham Singhal Date: Sun, 1 Mar 2026 12:57:13 +0530 Subject: [PATCH 15/15] Remove hardcoded value --- datafusion/proto/src/logical_plan/file_formats.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/proto/src/logical_plan/file_formats.rs b/datafusion/proto/src/logical_plan/file_formats.rs index 4b5749be157cd..eccf01303140a 100644 --- a/datafusion/proto/src/logical_plan/file_formats.rs +++ b/datafusion/proto/src/logical_plan/file_formats.rs @@ -526,7 +526,7 @@ mod parquet { max_predicate_cache_size: proto.max_predicate_cache_size_opt.as_ref().map(|opt| match opt { parquet_options::MaxPredicateCacheSizeOpt::MaxPredicateCacheSize(size) => *size as usize, }), - field_id_enabled: false, + field_id_enabled: proto.field_id_enabled, } } }