diff --git a/Cargo.lock b/Cargo.lock index b9dbd37463dd6..9840a3047f6ba 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2221,6 +2221,8 @@ dependencies = [ "dashmap", "datafusion-common", "datafusion-expr", + "datafusion-physical-expr", + "datafusion-physical-expr-common", "futures", "insta", "log", diff --git a/datafusion/catalog-listing/src/table.rs b/datafusion/catalog-listing/src/table.rs index 9fb2dd2dce29c..8e9a16ddcbc3f 100644 --- a/datafusion/catalog-listing/src/table.rs +++ b/datafusion/catalog-listing/src/table.rs @@ -34,7 +34,9 @@ use datafusion_datasource::schema_adapter::SchemaAdapterFactory; use datafusion_datasource::{ ListingTableUrl, PartitionedFile, TableSchema, compute_all_files_statistics, }; -use datafusion_execution::cache::cache_manager::FileStatisticsCache; +use datafusion_execution::cache::cache_manager::{ + CachedFileMetadata, FileStatisticsCache, +}; use datafusion_execution::cache::cache_unit::DefaultFileStatisticsCache; use datafusion_expr::dml::InsertOp; use datafusion_expr::execution_props::ExecutionProps; @@ -336,16 +338,103 @@ impl ListingTable { self.options.format.file_source(table_schema) } - /// If file_sort_order is specified, creates the appropriate physical expressions + /// Creates output ordering from user-specified file_sort_order or derives + /// from file orderings when user doesn't specify. + /// + /// If user specified `file_sort_order`, that takes precedence. + /// Otherwise, attempts to derive common ordering from file orderings in + /// the provided file groups. pub fn try_create_output_ordering( &self, execution_props: &ExecutionProps, + file_groups: &[FileGroup], ) -> datafusion_common::Result> { - create_lex_ordering( - &self.table_schema, - &self.options.file_sort_order, - execution_props, - ) + // If user specified sort order, use that + if !self.options.file_sort_order.is_empty() { + return create_lex_ordering( + &self.table_schema, + &self.options.file_sort_order, + execution_props, + ); + } + if let Some(ordering) = derive_common_ordering_from_files(file_groups) { + return Ok(vec![ordering]); + } + Ok(vec![]) + } +} + +/// Derives a common ordering from file orderings across all file groups. +/// +/// Returns the common ordering if all files have compatible orderings, +/// otherwise returns None. +/// +/// The function finds the longest common prefix among all file orderings. +/// For example, if files have orderings `[a, b, c]` and `[a, b]`, the common +/// ordering is `[a, b]`. +fn derive_common_ordering_from_files(file_groups: &[FileGroup]) -> Option { + enum CurrentOrderingState { + /// Initial state before processing any files + FirstFile, + /// Some common ordering found so far + SomeOrdering(LexOrdering), + /// No files have ordering + NoOrdering, + } + let mut state = CurrentOrderingState::FirstFile; + + // Collect file orderings and track counts + for group in file_groups { + for file in group.iter() { + state = match (&state, &file.ordering) { + // If this is the first file with ordering, set it as current + (CurrentOrderingState::FirstFile, Some(ordering)) => { + CurrentOrderingState::SomeOrdering(ordering.clone()) + } + (CurrentOrderingState::FirstFile, None) => { + CurrentOrderingState::NoOrdering + } + // If we have an existing ordering, find common prefix with new ordering + (CurrentOrderingState::SomeOrdering(current), Some(ordering)) => { + // Find common prefix between current and new ordering + let prefix_len = current + .as_ref() + .iter() + .zip(ordering.as_ref().iter()) + .take_while(|(a, b)| a == b) + .count(); + if prefix_len == 0 { + log::trace!( + "Cannot derive common ordering: no common prefix between orderings {current:?} and {ordering:?}" + ); + return None; + } else { + let ordering = + LexOrdering::new(current.as_ref()[..prefix_len].to_vec()) + .expect("prefix_len > 0, so ordering must be valid"); + CurrentOrderingState::SomeOrdering(ordering) + } + } + // If one file has ordering and another doesn't, no common ordering + // Return None and log a trace message explaining why + (CurrentOrderingState::SomeOrdering(ordering), None) + | (CurrentOrderingState::NoOrdering, Some(ordering)) => { + log::trace!( + "Cannot derive common ordering: some files have ordering {ordering:?}, others don't" + ); + return None; + } + // Both have no ordering, remain in NoOrdering state + (CurrentOrderingState::NoOrdering, None) => { + CurrentOrderingState::NoOrdering + } + }; + } + } + + match state { + CurrentOrderingState::SomeOrdering(ordering) => Some(ordering), + _ => None, } } @@ -438,7 +527,10 @@ impl TableProvider for ListingTable { return Ok(ScanResult::new(Arc::new(EmptyExec::new(projected_schema)))); } - let output_ordering = self.try_create_output_ordering(state.execution_props())?; + let output_ordering = self.try_create_output_ordering( + state.execution_props(), + &partitioned_file_lists, + )?; match state .config_options() .execution @@ -581,7 +673,8 @@ impl TableProvider for ListingTable { file_extension: self.options().format.get_ext(), }; - let orderings = self.try_create_output_ordering(state.execution_props())?; + // For writes, we only use user-specified ordering (no file groups to derive from) + let orderings = self.try_create_output_ordering(state.execution_props(), &[])?; // It is sufficient to pass only one of the equivalent orderings: let order_requirements = orderings.into_iter().next().map(Into::into); @@ -630,16 +723,19 @@ impl ListingTable { let meta_fetch_concurrency = ctx.config_options().execution.meta_fetch_concurrency; let file_list = stream::iter(file_list).flatten_unordered(meta_fetch_concurrency); - // collect the statistics if required by the config + // collect the statistics and ordering if required by the config let files = file_list .map(|part_file| async { let part_file = part_file?; - let statistics = if self.options.collect_stat { - self.do_collect_statistics(ctx, &store, &part_file).await? + let (statistics, ordering) = if self.options.collect_stat { + self.do_collect_statistics_and_ordering(ctx, &store, &part_file) + .await? } else { - Arc::new(Statistics::new_unknown(&self.file_schema)) + (Arc::new(Statistics::new_unknown(&self.file_schema)), None) }; - Ok(part_file.with_statistics(statistics)) + Ok(part_file + .with_statistics(statistics) + .with_ordering(ordering)) }) .boxed() .buffer_unordered(ctx.config_options().execution.meta_fetch_concurrency); @@ -694,42 +790,46 @@ impl ListingTable { }) } - /// Collects statistics for a given partitioned file. + /// Collects statistics and ordering for a given partitioned file. /// - /// This method first checks if the statistics for the given file are already cached. - /// If they are, it returns the cached statistics. - /// If they are not, it infers the statistics from the file and stores them in the cache. - async fn do_collect_statistics( + /// This method checks if both statistics and ordering are cached. + /// If both are cached, returns them without any file access. + /// If only statistics are cached, infers ordering and caches it. + /// If neither is cached, infers both in a single metadata read. + async fn do_collect_statistics_and_ordering( &self, ctx: &dyn Session, store: &Arc, part_file: &PartitionedFile, - ) -> datafusion_common::Result> { - match self - .collected_statistics - .get_with_extra(&part_file.object_meta.location, &part_file.object_meta) + ) -> datafusion_common::Result<(Arc, Option)> { + let path = &part_file.object_meta.location; + let meta = &part_file.object_meta; + + // Check if statistics and ordering are cached and valid + if let Some(cached) = self.collected_statistics.get(path) + && cached.is_valid_for(meta) { - Some(statistics) => Ok(statistics), - None => { - let statistics = self - .options - .format - .infer_stats( - ctx, - store, - Arc::clone(&self.file_schema), - &part_file.object_meta, - ) - .await?; - let statistics = Arc::new(statistics); - self.collected_statistics.put_with_extra( - &part_file.object_meta.location, - Arc::clone(&statistics), - &part_file.object_meta, - ); - Ok(statistics) - } + return Ok((cached.statistics.clone(), cached.ordering.clone())); } + + // Cache miss or invalid: fetch both statistics and ordering in a single metadata read + let file_meta = self + .options + .format + .infer_stats_and_ordering(ctx, store, Arc::clone(&self.file_schema), meta) + .await?; + + let statistics = Arc::new(file_meta.statistics); + self.collected_statistics.put( + path, + CachedFileMetadata::new( + meta.clone(), + Arc::clone(&statistics), + file_meta.ordering.clone(), + ), + ); + + Ok((statistics, file_meta.ordering)) } } @@ -805,3 +905,157 @@ async fn get_files_with_limit( let inexact_stats = all_files.next().await.is_some(); Ok((file_group, inexact_stats)) } + +#[cfg(test)] +mod tests { + use super::*; + use arrow::compute::SortOptions; + use datafusion_physical_expr::expressions::Column; + use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; + use std::sync::Arc; + + /// Helper to create a PhysicalSortExpr + fn sort_expr( + name: &str, + idx: usize, + descending: bool, + nulls_first: bool, + ) -> PhysicalSortExpr { + PhysicalSortExpr::new( + Arc::new(Column::new(name, idx)), + SortOptions { + descending, + nulls_first, + }, + ) + } + + /// Helper to create a LexOrdering (unwraps the Option) + fn lex_ordering(exprs: Vec) -> LexOrdering { + LexOrdering::new(exprs).expect("expected non-empty ordering") + } + + /// Helper to create a PartitionedFile with optional ordering + fn create_file(name: &str, ordering: Option) -> PartitionedFile { + PartitionedFile::new(name.to_string(), 1024).with_ordering(ordering) + } + + #[test] + fn test_derive_common_ordering_all_files_same_ordering() { + // All files have the same ordering → returns that ordering + let ordering = lex_ordering(vec![ + sort_expr("a", 0, false, true), + sort_expr("b", 1, true, false), + ]); + + let file_groups = vec![ + FileGroup::new(vec![ + create_file("f1.parquet", Some(ordering.clone())), + create_file("f2.parquet", Some(ordering.clone())), + ]), + FileGroup::new(vec![create_file("f3.parquet", Some(ordering.clone()))]), + ]; + + let result = derive_common_ordering_from_files(&file_groups); + + assert_eq!(result.unwrap().as_ref(), ordering.as_ref()); + } + + #[test] + fn test_derive_common_ordering_different_orderings() { + // Files have different orderings → returns empty + let ordering1 = lex_ordering(vec![sort_expr("a", 0, false, true)]); + let ordering2 = lex_ordering(vec![sort_expr("b", 1, false, true)]); + + let file_groups = vec![FileGroup::new(vec![ + create_file("f1.parquet", Some(ordering1)), + create_file("f2.parquet", Some(ordering2)), + ])]; + + let result = derive_common_ordering_from_files(&file_groups); + + assert!(result.is_none()); + } + + #[test] + fn test_derive_common_ordering_some_files_no_ordering() { + // Some files have no ordering → returns empty + let ordering = lex_ordering(vec![sort_expr("a", 0, false, true)]); + + let file_groups = vec![FileGroup::new(vec![ + create_file("f1.parquet", Some(ordering)), + create_file("f2.parquet", None), + ])]; + + let result = derive_common_ordering_from_files(&file_groups); + + assert!(result.is_none()); + } + + #[test] + fn test_derive_common_ordering_empty_file_groups() { + // Empty file groups → returns empty + let file_groups: Vec = vec![]; + + let result = derive_common_ordering_from_files(&file_groups); + + assert!(result.is_none()); + } + + #[test] + fn test_derive_common_ordering_empty_groups() { + // File groups with no files → returns empty + let file_groups = vec![FileGroup::new(vec![])]; + + let result = derive_common_ordering_from_files(&file_groups); + + assert!(result.is_none()); + } + + #[test] + fn test_derive_common_ordering_single_file() { + // Single file with ordering → returns that ordering + let ordering = lex_ordering(vec![sort_expr("a", 0, false, true)]); + + let file_groups = vec![FileGroup::new(vec![create_file( + "f1.parquet", + Some(ordering.clone()), + )])]; + + let result = derive_common_ordering_from_files(&file_groups); + + assert_eq!(result.unwrap().as_ref(), ordering.as_ref()); + } + + #[test] + fn test_derive_common_ordering_multiple_groups_same_ordering() { + // Multiple file groups, all with same ordering → returns that ordering + let ordering = lex_ordering(vec![sort_expr("a", 0, false, true)]); + + let file_groups = vec![ + FileGroup::new(vec![create_file("f1.parquet", Some(ordering.clone()))]), + FileGroup::new(vec![create_file("f2.parquet", Some(ordering.clone()))]), + FileGroup::new(vec![create_file("f3.parquet", Some(ordering.clone()))]), + ]; + + let result = derive_common_ordering_from_files(&file_groups); + + assert_eq!(result.unwrap().as_ref(), ordering.as_ref()); + } + + #[test] + fn test_derive_common_ordering_different_sort_options() { + // Same column but different sort options → returns empty + let ordering1 = lex_ordering(vec![sort_expr("a", 0, false, true)]); // ASC NULLS FIRST + let ordering2 = lex_ordering(vec![sort_expr("a", 0, true, true)]); // DESC NULLS FIRST + + let file_groups = vec![FileGroup::new(vec![ + create_file("f1.parquet", Some(ordering1)), + create_file("f2.parquet", Some(ordering2)), + ])]; + + let result = derive_common_ordering_from_files(&file_groups); + + assert!(result.is_none()); + } +} diff --git a/datafusion/core/src/datasource/file_format/mod.rs b/datafusion/core/src/datasource/file_format/mod.rs index 6bbb63f6a17ad..e8ecdfcead04d 100644 --- a/datafusion/core/src/datasource/file_format/mod.rs +++ b/datafusion/core/src/datasource/file_format/mod.rs @@ -80,6 +80,7 @@ pub(crate) mod test_util { range: None, statistics: None, extensions: None, + ordering: None, metadata_size_hint: None, }] .into(), diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 93d77e10ba23c..4e33f3cad51a4 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -347,7 +347,7 @@ mod tests { let table = ListingTable::try_new(config.clone()).expect("Creating the table"); let ordering_result = - table.try_create_output_ordering(state.execution_props()); + table.try_create_output_ordering(state.execution_props(), &[]); match (expected_result, ordering_result) { (Ok(expected), Ok(result)) => { diff --git a/datafusion/core/src/datasource/listing_table_factory.rs b/datafusion/core/src/datasource/listing_table_factory.rs index 3ca388af0c4c1..f401aa459e3e1 100644 --- a/datafusion/core/src/datasource/listing_table_factory.rs +++ b/datafusion/core/src/datasource/listing_table_factory.rs @@ -220,8 +220,9 @@ mod tests { datasource::file_format::csv::CsvFormat, execution::context::SessionContext, test_util::parquet_test_data, }; - use datafusion_execution::cache::CacheAccessor; - use datafusion_execution::cache::cache_manager::CacheManagerConfig; + use datafusion_execution::cache::cache_manager::{ + CacheManagerConfig, FileStatisticsCache, + }; use datafusion_execution::cache::cache_unit::DefaultFileStatisticsCache; use datafusion_execution::config::SessionConfig; use datafusion_execution::runtime_env::RuntimeEnvBuilder; diff --git a/datafusion/core/src/datasource/mod.rs b/datafusion/core/src/datasource/mod.rs index aefda64d39367..1f21d6a7e603a 100644 --- a/datafusion/core/src/datasource/mod.rs +++ b/datafusion/core/src/datasource/mod.rs @@ -113,14 +113,7 @@ mod tests { version: None, }; - let partitioned_file = PartitionedFile { - object_meta: meta, - partition_values: vec![], - range: None, - statistics: None, - extensions: None, - metadata_size_hint: None, - }; + let partitioned_file = PartitionedFile::new_from_meta(meta); let f1 = Field::new("id", DataType::Int32, true); let f2 = Field::new("extra_column", DataType::Utf8, true); diff --git a/datafusion/core/src/datasource/physical_plan/parquet.rs b/datafusion/core/src/datasource/physical_plan/parquet.rs index 4703b55ecc0de..ce2b05e6d3b61 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet.rs @@ -54,7 +54,7 @@ mod tests { use datafusion_datasource::source::DataSourceExec; use datafusion_datasource::file::FileSource; - use datafusion_datasource::{FileRange, PartitionedFile, TableSchema}; + use datafusion_datasource::{PartitionedFile, TableSchema}; use datafusion_datasource_parquet::source::ParquetSource; use datafusion_datasource_parquet::{ DefaultParquetFileReaderFactory, ParquetFileReaderFactory, ParquetFormat, @@ -1527,14 +1527,7 @@ mod tests { #[tokio::test] async fn parquet_exec_with_range() -> Result<()> { fn file_range(meta: &ObjectMeta, start: i64, end: i64) -> PartitionedFile { - PartitionedFile { - object_meta: meta.clone(), - partition_values: vec![], - range: Some(FileRange { start, end }), - statistics: None, - extensions: None, - metadata_size_hint: None, - } + PartitionedFile::new_from_meta(meta.clone()).with_range(start, end) } async fn assert_parquet_read( @@ -1616,21 +1609,15 @@ mod tests { .await .unwrap(); - let partitioned_file = PartitionedFile { - object_meta: meta, - partition_values: vec![ + let partitioned_file = PartitionedFile::new_from_meta(meta) + .with_partition_values(vec![ ScalarValue::from("2021"), ScalarValue::UInt8(Some(10)), ScalarValue::Dictionary( Box::new(DataType::UInt16), Box::new(ScalarValue::from("26")), ), - ], - range: None, - statistics: None, - extensions: None, - metadata_size_hint: None, - }; + ]); let expected_schema = Schema::new(vec![ Field::new("id", DataType::Int32, true), @@ -1711,20 +1698,13 @@ mod tests { .unwrap() .child("invalid.parquet"); - let partitioned_file = PartitionedFile { - object_meta: ObjectMeta { - location, - last_modified: Utc.timestamp_nanos(0), - size: 1337, - e_tag: None, - version: None, - }, - partition_values: vec![], - range: None, - statistics: None, - extensions: None, - metadata_size_hint: None, - }; + let partitioned_file = PartitionedFile::new_from_meta(ObjectMeta { + location, + last_modified: Utc.timestamp_nanos(0), + size: 1337, + e_tag: None, + version: None, + }); let file_schema = Arc::new(Schema::empty()); let config = FileScanConfigBuilder::new( @@ -2376,36 +2356,22 @@ mod tests { ); let config = FileScanConfigBuilder::new(store_url, source) .with_file( - PartitionedFile { - object_meta: ObjectMeta { - location: Path::from(name_1), - last_modified: Utc::now(), - size: total_size_1, - e_tag: None, - version: None, - }, - partition_values: vec![], - range: None, - statistics: None, - extensions: None, - metadata_size_hint: None, - } - .with_metadata_size_hint(123), - ) - .with_file(PartitionedFile { - object_meta: ObjectMeta { - location: Path::from(name_2), + PartitionedFile::new_from_meta(ObjectMeta { + location: Path::from(name_1), last_modified: Utc::now(), - size: total_size_2, + size: total_size_1, e_tag: None, version: None, - }, - partition_values: vec![], - range: None, - statistics: None, - extensions: None, - metadata_size_hint: None, - }) + }) + .with_metadata_size_hint(123), + ) + .with_file(PartitionedFile::new_from_meta(ObjectMeta { + location: Path::from(name_2), + last_modified: Utc::now(), + size: total_size_2, + e_tag: None, + version: None, + })) .build(); let exec = DataSourceExec::from_data_source(config); diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index cc7d534776d7e..1c7773e9e5ca7 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -573,6 +573,9 @@ impl DefaultPhysicalPlanner { file_extension, }; + // Note that because the input is an arbitrary execution plan, this ordering may involve expressions + // e.g. `ORDER BY random()`. + // In this case the file format probably cannot encode the ordering and it will be ignored. let ordering = input_exec.properties().output_ordering().cloned(); sink_format diff --git a/datafusion/core/src/test_util/parquet.rs b/datafusion/core/src/test_util/parquet.rs index 44e884c23a681..13a41b385110c 100644 --- a/datafusion/core/src/test_util/parquet.rs +++ b/datafusion/core/src/test_util/parquet.rs @@ -160,16 +160,8 @@ impl TestParquetFile { .with_table_parquet_options(parquet_options.clone()), ); let scan_config_builder = - FileScanConfigBuilder::new(self.object_store_url.clone(), source).with_file( - PartitionedFile { - object_meta: self.object_meta.clone(), - partition_values: vec![], - range: None, - statistics: None, - extensions: None, - metadata_size_hint: None, - }, - ); + FileScanConfigBuilder::new(self.object_store_url.clone(), source) + .with_file(PartitionedFile::new_from_meta(self.object_meta.clone())); let df_schema = Arc::clone(&self.schema).to_dfschema_ref()?; @@ -229,3 +221,91 @@ impl TestParquetFile { self.path.as_path() } } + +/// Specification for a sorting column in a Parquet file. +/// +/// This is used by [`create_sorted_parquet_file`] to define the sort order +/// when creating test Parquet files with sorting metadata. +#[derive(Debug, Clone)] +pub struct SortColumnSpec { + /// The column index in the schema (0-based) + pub column_idx: usize, + /// If true, the column is sorted in descending order + pub descending: bool, + /// If true, nulls come before non-null values + pub nulls_first: bool, +} + +impl SortColumnSpec { + /// Create a new sort column specification + pub fn new(column_idx: usize, descending: bool, nulls_first: bool) -> Self { + Self { + column_idx, + descending, + nulls_first, + } + } + + /// Create an ascending, nulls-first sort column + pub fn asc_nulls_first(column_idx: usize) -> Self { + Self::new(column_idx, false, true) + } + + /// Create an ascending, nulls-last sort column + pub fn asc_nulls_last(column_idx: usize) -> Self { + Self::new(column_idx, false, false) + } + + /// Create a descending, nulls-first sort column + pub fn desc_nulls_first(column_idx: usize) -> Self { + Self::new(column_idx, true, true) + } + + /// Create a descending, nulls-last sort column + pub fn desc_nulls_last(column_idx: usize) -> Self { + Self::new(column_idx, true, false) + } +} + +/// Creates a test Parquet file with sorting_columns metadata. +/// +/// This is useful for testing ordering inference from Parquet files. +/// +/// # Arguments +/// * `path` - The path where the Parquet file will be written +/// * `batches` - The record batches to write to the file +/// * `sorting_columns` - The sorting column specifications (defines the sort order) +/// +/// # Example +/// ```ignore +/// use datafusion::test_util::parquet::{create_sorted_parquet_file, SortColumnSpec}; +/// +/// let batches = vec![batch1, batch2]; +/// let sorting = vec![ +/// SortColumnSpec::asc_nulls_first(0), // First column ascending +/// SortColumnSpec::desc_nulls_last(1), // Second column descending +/// ]; +/// let test_file = create_sorted_parquet_file(path, batches, sorting)?; +/// ``` +pub fn create_sorted_parquet_file( + path: PathBuf, + batches: impl IntoIterator, + sorting_columns: Vec, +) -> Result { + use parquet::file::metadata::SortingColumn; + + let parquet_sorting_columns: Vec = sorting_columns + .into_iter() + .map(|spec| SortingColumn { + column_idx: spec.column_idx as i32, + descending: spec.descending, + nulls_first: spec.nulls_first, + }) + .collect(); + + let props = WriterProperties::builder() + .set_sorting_columns(Some(parquet_sorting_columns)) + .build(); + + TestParquetFile::try_new(path, props, batches) +} diff --git a/datafusion/core/tests/parquet/custom_reader.rs b/datafusion/core/tests/parquet/custom_reader.rs index 31ec6efd19510..25f69d2975eac 100644 --- a/datafusion/core/tests/parquet/custom_reader.rs +++ b/datafusion/core/tests/parquet/custom_reader.rs @@ -69,13 +69,9 @@ async fn route_data_access_ops_to_parquet_file_reader_factory() { store_parquet_in_memory(vec![batch]).await; let file_group = parquet_files_meta .into_iter() - .map(|meta| PartitionedFile { - object_meta: meta, - partition_values: vec![], - range: None, - statistics: None, - extensions: Some(Arc::new(String::from(EXPECTED_USER_DEFINED_METADATA))), - metadata_size_hint: None, + .map(|meta| { + PartitionedFile::new_from_meta(meta) + .with_extensions(Arc::new(String::from(EXPECTED_USER_DEFINED_METADATA))) }) .collect(); diff --git a/datafusion/core/tests/parquet/mod.rs b/datafusion/core/tests/parquet/mod.rs index 35b5918d9e8bf..44c9a2393e3d8 100644 --- a/datafusion/core/tests/parquet/mod.rs +++ b/datafusion/core/tests/parquet/mod.rs @@ -50,6 +50,7 @@ mod expr_adapter; mod external_access_plan; mod file_statistics; mod filter_pushdown; +mod ordering; mod page_pruning; mod row_group_pruning; mod schema; diff --git a/datafusion/core/tests/parquet/ordering.rs b/datafusion/core/tests/parquet/ordering.rs new file mode 100644 index 0000000000000..d1fdb16052288 --- /dev/null +++ b/datafusion/core/tests/parquet/ordering.rs @@ -0,0 +1,249 @@ +// 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. + +//! Tests for ordering inference from Parquet sorting_columns metadata + +use std::sync::Arc; + +use arrow::array::Int32Array; +use arrow::datatypes::{DataType, Field, Schema}; +use arrow::record_batch::RecordBatch; +use datafusion::prelude::SessionContext; +use datafusion::test_util::parquet::{SortColumnSpec, create_sorted_parquet_file}; +use datafusion_common::Result; +use tempfile::tempdir; + +/// Test that ordering is inferred from Parquet sorting_columns metadata +#[tokio::test] +async fn test_parquet_ordering_inference() -> Result<()> { + // Create a schema with two integer columns + let schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + ])); + + // Create sorted test data (sorted by 'a' ascending) + 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 a temp directory for the test file + let tmp_dir = tempdir()?; + let file_path = tmp_dir.path().join("sorted.parquet"); + + // Create the Parquet file with sorting metadata + let sorting = vec![ + SortColumnSpec::asc_nulls_first(0), // column 'a' ASC NULLS FIRST + ]; + let _test_file = create_sorted_parquet_file(file_path.clone(), vec![batch], sorting)?; + + // Register the file as a table and query it + let ctx = SessionContext::new(); + ctx.register_parquet( + "sorted_table", + file_path.to_str().unwrap(), + Default::default(), + ) + .await?; + + // Verify the table is registered and can be queried + let df = ctx.sql("SELECT * FROM sorted_table ORDER BY a").await?; + let results = df.collect().await?; + + // Should have 1 batch with 5 rows + assert_eq!(results.len(), 1); + assert_eq!(results[0].num_rows(), 5); + + // Verify data is correct + let a_col = results[0] + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(a_col.value(0), 1); + assert_eq!(a_col.value(4), 5); + + Ok(()) +} + +/// Test that multi-column sorting metadata is preserved +#[tokio::test] +async fn test_parquet_multi_column_ordering_inference() -> Result<()> { + let schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("c", DataType::Int32, true), + ])); + + // Create test data sorted by 'a' ASC, then 'b' DESC + let batch = RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(Int32Array::from(vec![1, 1, 2, 2, 3])), + Arc::new(Int32Array::from(vec![20, 10, 40, 30, 50])), + Arc::new(Int32Array::from(vec![100, 200, 300, 400, 500])), + ], + )?; + + let tmp_dir = tempdir()?; + let file_path = tmp_dir.path().join("multi_sorted.parquet"); + + // Create the file with multi-column sorting metadata + let sorting = vec![ + SortColumnSpec::asc_nulls_first(0), // 'a' ASC NULLS FIRST + SortColumnSpec::desc_nulls_last(1), // 'b' DESC NULLS LAST + ]; + let _test_file = create_sorted_parquet_file(file_path.clone(), vec![batch], sorting)?; + + let ctx = SessionContext::new(); + ctx.register_parquet( + "multi_sorted", + file_path.to_str().unwrap(), + Default::default(), + ) + .await?; + + let df = ctx.sql("SELECT a, b FROM multi_sorted").await?; + let results = df.collect().await?; + + assert_eq!(results.len(), 1); + assert_eq!(results[0].num_rows(), 5); + + Ok(()) +} + +/// Test file without sorting metadata +#[tokio::test] +async fn test_parquet_no_ordering_metadata() -> Result<()> { + use parquet::arrow::ArrowWriter; + use parquet::file::properties::WriterProperties; + use std::fs::File; + + let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, true)])); + + let batch = RecordBatch::try_new( + Arc::clone(&schema), + vec![Arc::new(Int32Array::from(vec![3, 1, 2]))], + )?; + + let tmp_dir = tempdir()?; + let file_path = tmp_dir.path().join("unsorted.parquet"); + + // Create file WITHOUT sorting metadata + let file = File::create(&file_path)?; + let props = WriterProperties::builder().build(); + let mut writer = ArrowWriter::try_new(file, Arc::clone(&schema), Some(props))?; + writer.write(&batch)?; + writer.close()?; + + let ctx = SessionContext::new(); + ctx.register_parquet("unsorted", file_path.to_str().unwrap(), Default::default()) + .await?; + + let df = ctx.sql("SELECT * FROM unsorted").await?; + let results = df.collect().await?; + + assert_eq!(results.len(), 1); + assert_eq!(results[0].num_rows(), 3); + + Ok(()) +} + +/// Test that CREATE EXTERNAL TABLE WITH ORDER writes sorting_columns to Parquet metadata +#[tokio::test] +async fn test_create_table_with_order_writes_sorting_columns() -> Result<()> { + use parquet::file::reader::FileReader; + use parquet::file::serialized_reader::SerializedFileReader; + use std::fs::File; + + let ctx = SessionContext::new(); + let tmp_dir = tempdir()?; + let table_path = tmp_dir.path().join("sorted_table"); + std::fs::create_dir_all(&table_path)?; + + // Create external table with ordering + let create_table_sql = format!( + "CREATE EXTERNAL TABLE sorted_data (a INT, b VARCHAR) \ + STORED AS PARQUET \ + LOCATION '{}' \ + WITH ORDER (a ASC NULLS FIRST, b DESC NULLS LAST)", + table_path.display() + ); + ctx.sql(&create_table_sql).await?; + + // Insert sorted data + ctx.sql("INSERT INTO sorted_data VALUES (1, 'x'), (2, 'y'), (3, 'z')") + .await? + .collect() + .await?; + + // Find the parquet file that was written + let parquet_files: Vec<_> = std::fs::read_dir(&table_path)? + .filter_map(|e| e.ok()) + .filter(|e| e.path().extension().is_some_and(|ext| ext == "parquet")) + .collect(); + + assert!( + !parquet_files.is_empty(), + "Expected at least one parquet file in {}", + table_path.display() + ); + + // Read the parquet file and verify sorting_columns metadata + let file = File::open(parquet_files[0].path())?; + let reader = SerializedFileReader::new(file)?; + let metadata = reader.metadata(); + + // Check that row group has sorting_columns + let row_group = metadata.row_group(0); + let sorting_columns = row_group.sorting_columns(); + + assert!( + sorting_columns.is_some(), + "Expected sorting_columns in row group metadata" + ); + let sorting = sorting_columns.unwrap(); + assert_eq!(sorting.len(), 2, "Expected 2 sorting columns"); + + // First column: a ASC NULLS FIRST (column_idx = 0) + assert_eq!(sorting[0].column_idx, 0, "First sort column should be 'a'"); + assert!( + !sorting[0].descending, + "First column should be ASC (descending=false)" + ); + assert!( + sorting[0].nulls_first, + "First column should have NULLS FIRST" + ); + + // Second column: b DESC NULLS LAST (column_idx = 1) + assert_eq!(sorting[1].column_idx, 1, "Second sort column should be 'b'"); + assert!( + sorting[1].descending, + "Second column should be DESC (descending=true)" + ); + assert!( + !sorting[1].nulls_first, + "Second column should have NULLS LAST" + ); + + Ok(()) +} diff --git a/datafusion/core/tests/parquet/page_pruning.rs b/datafusion/core/tests/parquet/page_pruning.rs index 17392974b63a8..7eb39bfe78305 100644 --- a/datafusion/core/tests/parquet/page_pruning.rs +++ b/datafusion/core/tests/parquet/page_pruning.rs @@ -67,14 +67,7 @@ async fn get_parquet_exec( .await .unwrap(); - let partitioned_file = PartitionedFile { - object_meta: meta, - partition_values: vec![], - range: None, - statistics: None, - extensions: None, - metadata_size_hint: None, - }; + let partitioned_file = PartitionedFile::new_from_meta(meta); let df_schema = schema.clone().to_dfschema().unwrap(); let execution_props = ExecutionProps::new(); diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs index d6357fdf6bc7d..bd0e660acd60e 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs @@ -1829,7 +1829,7 @@ async fn test_topk_dynamic_filter_pushdown_integration() { COPY ( SELECT 1372708800 + value AS t FROM generate_series(0, 99999) - ORDER BY t + ORDER BY t + 1 ) TO 'memory:///1.parquet' STORED AS PARQUET; ", diff --git a/datafusion/datasource-parquet/src/file_format.rs b/datafusion/datasource-parquet/src/file_format.rs index 5e482382be687..0ca74f95b06df 100644 --- a/datafusion/datasource-parquet/src/file_format.rs +++ b/datafusion/datasource-parquet/src/file_format.rs @@ -34,7 +34,7 @@ use datafusion_datasource::write::{ ObjectWriterBuilder, SharedBuffer, get_writer_schema, }; -use datafusion_datasource::file_format::{FileFormat, FileFormatFactory}; +use datafusion_datasource::file_format::{FileFormat, FileFormatFactory, FileMeta}; use datafusion_datasource::write::demux::DemuxedStreamReceiver; use arrow::datatypes::{DataType, Field, FieldRef}; @@ -54,11 +54,11 @@ use datafusion_datasource::sink::{DataSink, DataSinkExec}; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryPool, MemoryReservation}; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_expr::dml::InsertOp; -use datafusion_physical_expr_common::sort_expr::LexRequirement; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; use datafusion_physical_plan::{DisplayAs, DisplayFormatType, ExecutionPlan}; use datafusion_session::Session; -use crate::metadata::DFParquetMetadata; +use crate::metadata::{DFParquetMetadata, lex_ordering_to_sorting_columns}; use crate::reader::CachedParquetFileReaderFactory; use crate::source::{ParquetSource, parse_coerce_int96_string}; use async_trait::async_trait; @@ -81,7 +81,7 @@ use parquet::basic::Type; #[cfg(feature = "parquet_encryption")] use parquet::encryption::encrypt::FileEncryptionProperties; use parquet::errors::ParquetError; -use parquet::file::metadata::ParquetMetaData; +use parquet::file::metadata::{ParquetMetaData, SortingColumn}; use parquet::file::properties::{WriterProperties, WriterPropertiesBuilder}; use parquet::file::writer::SerializedFileWriter; use parquet::schema::types::SchemaDescriptor; @@ -449,6 +449,48 @@ impl FileFormat for ParquetFormat { .await } + async fn infer_ordering( + &self, + state: &dyn Session, + store: &Arc, + table_schema: SchemaRef, + object: &ObjectMeta, + ) -> Result> { + let file_decryption_properties = + get_file_decryption_properties(state, &self.options, &object.location) + .await?; + let file_metadata_cache = + state.runtime_env().cache_manager.get_file_metadata_cache(); + let metadata = DFParquetMetadata::new(store, object) + .with_metadata_size_hint(self.metadata_size_hint()) + .with_decryption_properties(file_decryption_properties) + .with_file_metadata_cache(Some(file_metadata_cache)) + .fetch_metadata() + .await?; + DFParquetMetadata::ordering_from_parquet_metadata(&metadata, &table_schema) + } + + async fn infer_stats_and_ordering( + &self, + state: &dyn Session, + store: &Arc, + table_schema: SchemaRef, + object: &ObjectMeta, + ) -> Result { + let file_decryption_properties = + get_file_decryption_properties(state, &self.options, &object.location) + .await?; + let file_metadata_cache = + state.runtime_env().cache_manager.get_file_metadata_cache(); + let (statistics, ordering) = DFParquetMetadata::new(store, object) + .with_metadata_size_hint(self.metadata_size_hint()) + .with_decryption_properties(file_decryption_properties) + .with_file_metadata_cache(Some(file_metadata_cache)) + .fetch_statistics_and_ordering(&table_schema) + .await?; + Ok(FileMeta::new(statistics).with_ordering(ordering)) + } + async fn create_physical_plan( &self, state: &dyn Session, @@ -500,7 +542,22 @@ impl FileFormat for ParquetFormat { return not_impl_err!("Overwrites are not implemented yet for Parquet"); } - let sink = Arc::new(ParquetSink::new(conf, self.options.clone())); + // Convert ordering requirements to Parquet SortingColumns for file metadata + let sorting_columns = if let Some(ref requirements) = order_requirements { + let ordering: LexOrdering = requirements.clone().into(); + // In cases like `COPY (... ORDER BY ...) TO ...` the ORDER BY clause + // may not be compatible with Parquet sorting columns (e.g. ordering on `random()`). + // So if we cannot create a Parquet sorting column from the ordering requirement, + // we skip setting sorting columns on the Parquet sink. + lex_ordering_to_sorting_columns(&ordering).ok() + } else { + None + }; + + let sink = Arc::new( + ParquetSink::new(conf, self.options.clone()) + .with_sorting_columns(sorting_columns), + ); Ok(Arc::new(DataSinkExec::new(input, sink, order_requirements)) as _) } @@ -1088,6 +1145,8 @@ pub struct ParquetSink { /// File metadata from successfully produced parquet files. The Mutex is only used /// to allow inserting to HashMap from behind borrowed reference in DataSink::write_all. written: Arc>>, + /// Optional sorting columns to write to Parquet metadata + sorting_columns: Option>, } impl Debug for ParquetSink { @@ -1119,9 +1178,19 @@ impl ParquetSink { config, parquet_options, written: Default::default(), + sorting_columns: None, } } + /// Set sorting columns for the Parquet file metadata. + pub fn with_sorting_columns( + mut self, + sorting_columns: Option>, + ) -> Self { + self.sorting_columns = sorting_columns; + self + } + /// Retrieve the file metadata for the written files, keyed to the path /// which may be partitioned (in the case of hive style partitioning). pub fn written(&self) -> HashMap { @@ -1145,6 +1214,12 @@ impl ParquetSink { } let mut builder = WriterPropertiesBuilder::try_from(&parquet_opts)?; + + // Set sorting columns if configured + if let Some(ref sorting_columns) = self.sorting_columns { + builder = builder.set_sorting_columns(Some(sorting_columns.clone())); + } + builder = set_writer_encryption_properties( builder, runtime, diff --git a/datafusion/datasource-parquet/src/metadata.rs b/datafusion/datasource-parquet/src/metadata.rs index 8b11ba64ae7f1..f89ee181f56dd 100644 --- a/datafusion/datasource-parquet/src/metadata.rs +++ b/datafusion/datasource-parquet/src/metadata.rs @@ -22,6 +22,7 @@ use crate::{ ObjectStoreFetch, apply_file_schema_type_coercions, coerce_int96_to_resolution, }; use arrow::array::{ArrayRef, BooleanArray}; +use arrow::compute::SortOptions; use arrow::compute::and; use arrow::compute::kernels::cmp::eq; use arrow::compute::sum; @@ -31,8 +32,12 @@ use datafusion_common::stats::Precision; use datafusion_common::{ ColumnStatistics, DataFusionError, Result, ScalarValue, Statistics, }; -use datafusion_execution::cache::cache_manager::{FileMetadata, FileMetadataCache}; +use datafusion_execution::cache::cache_manager::{ + CachedFileMetadataEntry, FileMetadata, FileMetadataCache, +}; use datafusion_functions_aggregate_common::min_max::{MaxAccumulator, MinAccumulator}; +use datafusion_physical_expr::expressions::Column; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; use datafusion_physical_plan::Accumulator; use log::debug; use object_store::path::Path; @@ -41,6 +46,7 @@ use parquet::arrow::arrow_reader::statistics::StatisticsConverter; use parquet::arrow::{parquet_column, parquet_to_arrow_schema}; use parquet::file::metadata::{ PageIndexPolicy, ParquetMetaData, ParquetMetaDataReader, RowGroupMetaData, + SortingColumn, }; use parquet::schema::types::SchemaDescriptor; use std::any::Any; @@ -127,9 +133,13 @@ impl<'a> DFParquetMetadata<'a> { if cache_metadata && let Some(parquet_metadata) = file_metadata_cache .as_ref() - .and_then(|file_metadata_cache| file_metadata_cache.get(object_meta)) - .and_then(|file_metadata| { - file_metadata + .and_then(|file_metadata_cache| { + file_metadata_cache.get(&object_meta.location) + }) + .filter(|cached| cached.is_valid_for(object_meta)) + .and_then(|cached| { + cached + .file_metadata .as_any() .downcast_ref::() .map(|cached_parquet_metadata| { @@ -163,8 +173,11 @@ impl<'a> DFParquetMetadata<'a> { if cache_metadata && let Some(file_metadata_cache) = file_metadata_cache { file_metadata_cache.put( - object_meta, - Arc::new(CachedParquetMetaData::new(Arc::clone(&metadata))), + &object_meta.location, + CachedFileMetadataEntry::new( + (*object_meta).clone(), + Arc::new(CachedParquetMetaData::new(Arc::clone(&metadata))), + ), ); } @@ -208,6 +221,26 @@ impl<'a> DFParquetMetadata<'a> { Self::statistics_from_parquet_metadata(&metadata, table_schema) } + /// Fetch both statistics and ordering from Parquet metadata in a single metadata fetch. + /// + /// This is more efficient than calling [`Self::fetch_statistics`] and then separately + /// extracting ordering, as it only fetches the metadata once. + /// + /// # Returns + /// A tuple of (Statistics, `Option`) where the ordering is `None` if: + /// - No row groups have sorting_columns + /// - Row groups have inconsistent sorting_columns + /// - Sorting columns cannot be mapped to the Arrow schema + pub async fn fetch_statistics_and_ordering( + &self, + table_schema: &SchemaRef, + ) -> Result<(Statistics, Option)> { + let metadata = self.fetch_metadata().await?; + let statistics = Self::statistics_from_parquet_metadata(&metadata, table_schema)?; + let ordering = Self::ordering_from_parquet_metadata(&metadata, table_schema)?; + Ok((statistics, ordering)) + } + /// Convert statistics in [`ParquetMetaData`] into [`Statistics`] using [`StatisticsConverter`] /// /// The statistics are calculated for each column in the table schema @@ -373,6 +406,194 @@ impl<'a> DFParquetMetadata<'a> { Ok(statistics) } + + /// Extract LexOrdering from Parquet sorting_columns metadata. + /// + /// Returns `Ok(None)` if: + /// - No row groups exist + /// - No row group has sorting_columns + /// - Row groups have inconsistent sorting_columns + /// - A sorting column cannot be mapped to the Arrow schema + /// + /// # Arguments + /// * `metadata` - The Parquet file metadata + /// * `arrow_schema` - The Arrow schema to map column indices to + pub(crate) fn ordering_from_parquet_metadata( + metadata: &ParquetMetaData, + arrow_schema: &SchemaRef, + ) -> Result> { + let row_groups = metadata.row_groups(); + if row_groups.is_empty() { + return Ok(None); + } + + // Get sorting_columns from first row group + let first_sorting = match row_groups[0].sorting_columns() { + Some(cols) if !cols.is_empty() => cols, + _ => return Ok(None), + }; + + // Verify all row groups have identical sorting_columns + for rg in &row_groups[1..] { + match rg.sorting_columns() { + Some(cols) if cols == first_sorting => {} + _ => { + debug!( + "Row groups have inconsistent sorting_columns, treating as unordered" + ); + return Ok(None); + } + } + } + + // Get the Parquet schema descriptor for column name lookup + let file_metadata = metadata.file_metadata(); + let parquet_schema = file_metadata.schema_descr(); + + // Convert Parquet schema to Arrow schema for column mapping + let parquet_arrow_schema = + parquet_to_arrow_schema(parquet_schema, file_metadata.key_value_metadata())?; + + // Convert each SortingColumn to PhysicalSortExpr + let sort_exprs: Vec = first_sorting + .iter() + .filter_map(|sorting_col| { + sorting_column_to_sort_expr( + sorting_col, + parquet_schema, + &parquet_arrow_schema, + arrow_schema, + ) + .ok() + .flatten() + }) + .collect(); + + // If we couldn't map any columns, return None + if sort_exprs.is_empty() { + return Ok(None); + } + + // If we couldn't map all columns, the ordering is incomplete + // Only return the ordering if we mapped all sorting columns + if sort_exprs.len() != first_sorting.len() { + debug!( + "Could only map {}/{} sorting columns to Arrow schema", + sort_exprs.len(), + first_sorting.len() + ); + return Ok(None); + } + + Ok(LexOrdering::new(sort_exprs)) + } +} + +/// Convert a Parquet SortingColumn to a PhysicalSortExpr. +/// +/// Returns `Ok(None)` if the column cannot be mapped to the Arrow schema. +fn sorting_column_to_sort_expr( + sorting_col: &SortingColumn, + parquet_schema: &SchemaDescriptor, + parquet_arrow_schema: &Schema, + arrow_schema: &SchemaRef, +) -> Result> { + let column_idx = sorting_col.column_idx as usize; + + // Get the column path from the Parquet schema + // The column_idx in SortingColumn refers to leaf columns + if column_idx >= parquet_schema.num_columns() { + debug!( + "SortingColumn column_idx {} out of bounds (schema has {} columns)", + column_idx, + parquet_schema.num_columns() + ); + return Ok(None); + } + + let parquet_column_desc = parquet_schema.column(column_idx); + let column_path = parquet_column_desc.path().string(); + + // Find the corresponding field in the Parquet-derived Arrow schema + let parquet_arrow_field_idx = parquet_arrow_schema + .fields() + .iter() + .position(|f| f.name() == &column_path); + + let arrow_field_name = match parquet_arrow_field_idx { + Some(idx) => parquet_arrow_schema.field(idx).name().clone(), + None => { + // For nested columns, the path might be different + // Try to find by the last component of the path + let last_component = + column_path.split('.').next_back().unwrap_or(&column_path); + if let Ok(field) = parquet_arrow_schema.field_with_name(last_component) { + field.name().clone() + } else { + debug!( + "Could not find Arrow field for Parquet column path: {column_path}" + ); + return Ok(None); + } + } + }; + + // Find the field index in the target Arrow schema + let arrow_field_idx = match arrow_schema.index_of(&arrow_field_name) { + Ok(idx) => idx, + Err(_) => { + debug!( + "Column '{arrow_field_name}' from Parquet sorting_columns not found in Arrow schema" + ); + return Ok(None); + } + }; + + // Create the Column expression + let column_expr = Arc::new(Column::new(&arrow_field_name, arrow_field_idx)); + + // Create the sort options + let sort_options = SortOptions { + descending: sorting_col.descending, + nulls_first: sorting_col.nulls_first, + }; + + Ok(Some(PhysicalSortExpr::new(column_expr, sort_options))) +} + +/// Convert a PhysicalSortExpr to a Parquet SortingColumn. +/// +/// Returns `Err` if the expression is not a simple column reference, +/// since Parquet's SortingColumn only supports column indices. +pub(crate) fn sort_expr_to_sorting_column( + sort_expr: &PhysicalSortExpr, +) -> Result { + let column = sort_expr + .expr + .as_any() + .downcast_ref::() + .ok_or_else(|| { + DataFusionError::Plan(format!( + "Parquet sorting_columns only supports simple column references, \ + but got expression: {}", + sort_expr.expr + )) + })?; + + Ok(SortingColumn { + column_idx: column.index() as i32, + descending: sort_expr.options.descending, + nulls_first: sort_expr.options.nulls_first, + }) +} + +/// Convert a LexOrdering to `Vec` for Parquet. +/// +/// Returns `Err` if any expression is not a simple column reference. +pub(crate) fn lex_ordering_to_sorting_columns( + ordering: &LexOrdering, +) -> Result> { + ordering.iter().map(sort_expr_to_sorting_column).collect() } /// Min/max aggregation can take Dictionary encode input but always produces unpacked @@ -617,9 +838,331 @@ impl FileMetadata for CachedParquetMetaData { mod tests { use super::*; use arrow::array::{ArrayRef, BooleanArray, Int32Array}; + use arrow::datatypes::Field; use datafusion_common::ScalarValue; + use parquet::basic::Type; + use parquet::file::metadata::{ + ColumnChunkMetaData, FileMetaData, ParquetMetaDataBuilder, RowGroupMetaData, + }; + use parquet::schema::types::SchemaDescriptor; + use parquet::schema::types::Type as SchemaType; use std::sync::Arc; + /// Create a test Parquet schema descriptor with given column names (all INT32) + fn create_test_schema_descr(column_names: &[&str]) -> Arc { + let fields: Vec> = column_names + .iter() + .map(|name| { + Arc::new( + SchemaType::primitive_type_builder(name, Type::INT32) + .build() + .unwrap(), + ) + }) + .collect(); + + let schema = SchemaType::group_type_builder("schema") + .with_fields(fields) + .build() + .unwrap(); + + Arc::new(SchemaDescriptor::new(Arc::new(schema))) + } + + /// Create a test Arrow schema with given column names (all Int32) + fn create_test_arrow_schema(column_names: &[&str]) -> SchemaRef { + let fields: Vec = column_names + .iter() + .map(|name| Field::new(*name, DataType::Int32, true)) + .collect(); + Arc::new(Schema::new(fields)) + } + + /// Create a RowGroupMetaData with the given sorting columns + fn create_row_group_with_sorting( + schema_descr: &Arc, + sorting_columns: Option>, + num_rows: i64, + ) -> RowGroupMetaData { + let columns: Vec = schema_descr + .columns() + .iter() + .map(|col| ColumnChunkMetaData::builder(col.clone()).build().unwrap()) + .collect(); + + let mut builder = RowGroupMetaData::builder(schema_descr.clone()) + .set_num_rows(num_rows) + .set_column_metadata(columns); + + if let Some(sorting) = sorting_columns { + builder = builder.set_sorting_columns(Some(sorting)); + } + + builder.build().unwrap() + } + + /// Create a ParquetMetaData with given row groups + fn create_parquet_metadata( + schema_descr: Arc, + row_groups: Vec, + ) -> ParquetMetaData { + let file_metadata = FileMetaData::new( + 1, // version + row_groups.iter().map(|rg| rg.num_rows()).sum::(), // num_rows + None, // created_by + None, // key_value_metadata + schema_descr, // schema_descr + None, // column_orders + ); + + ParquetMetaDataBuilder::new(file_metadata) + .set_row_groups(row_groups) + .build() + } + + #[test] + fn test_ordering_from_parquet_metadata_single_row_group_with_sorting() { + // Single row group with sorting_columns [a ASC NULLS FIRST, b DESC NULLS LAST] + let schema_descr = create_test_schema_descr(&["a", "b", "c"]); + let arrow_schema = create_test_arrow_schema(&["a", "b", "c"]); + + let sorting_columns = vec![ + SortingColumn { + column_idx: 0, + descending: false, + nulls_first: true, + }, // a ASC NULLS FIRST + SortingColumn { + column_idx: 1, + descending: true, + nulls_first: false, + }, // b DESC NULLS LAST + ]; + + let row_group = + create_row_group_with_sorting(&schema_descr, Some(sorting_columns), 1000); + let metadata = create_parquet_metadata(schema_descr, vec![row_group]); + + let result = + DFParquetMetadata::ordering_from_parquet_metadata(&metadata, &arrow_schema) + .unwrap(); + + assert!(result.is_some()); + let ordering = result.unwrap(); + assert_eq!(ordering.len(), 2); + + // Check first sort expr (a ASC NULLS FIRST) + let expr0 = &ordering[0]; + assert_eq!(expr0.expr.to_string(), "a@0"); + assert!(!expr0.options.descending); + assert!(expr0.options.nulls_first); + + // Check second sort expr (b DESC NULLS LAST) + let expr1 = &ordering[1]; + assert_eq!(expr1.expr.to_string(), "b@1"); + assert!(expr1.options.descending); + assert!(!expr1.options.nulls_first); + } + + #[test] + fn test_ordering_from_parquet_metadata_multiple_row_groups_identical_sorting() { + // Multiple row groups with identical sorting_columns + let schema_descr = create_test_schema_descr(&["a", "b"]); + let arrow_schema = create_test_arrow_schema(&["a", "b"]); + + let sorting_columns = vec![SortingColumn { + column_idx: 0, + descending: false, + nulls_first: true, + }]; // a ASC NULLS FIRST + + let row_group1 = create_row_group_with_sorting( + &schema_descr, + Some(sorting_columns.clone()), + 500, + ); + let row_group2 = create_row_group_with_sorting( + &schema_descr, + Some(sorting_columns.clone()), + 500, + ); + let row_group3 = + create_row_group_with_sorting(&schema_descr, Some(sorting_columns), 500); + + let metadata = create_parquet_metadata( + schema_descr, + vec![row_group1, row_group2, row_group3], + ); + + let result = + DFParquetMetadata::ordering_from_parquet_metadata(&metadata, &arrow_schema) + .unwrap(); + + assert!(result.is_some()); + let ordering = result.unwrap(); + assert_eq!(ordering.len(), 1); + + let expr0 = &ordering[0]; + assert_eq!(expr0.expr.to_string(), "a@0"); + assert!(!expr0.options.descending); + assert!(expr0.options.nulls_first); + } + + #[test] + fn test_ordering_from_parquet_metadata_multiple_row_groups_different_sorting() { + // Multiple row groups with DIFFERENT sorting_columns → should return None + let schema_descr = create_test_schema_descr(&["a", "b"]); + let arrow_schema = create_test_arrow_schema(&["a", "b"]); + + let sorting1 = vec![SortingColumn { + column_idx: 0, + descending: false, + nulls_first: true, + }]; // a ASC + let sorting2 = vec![SortingColumn { + column_idx: 1, + descending: false, + nulls_first: true, + }]; // b ASC (different column) + + let row_group1 = + create_row_group_with_sorting(&schema_descr, Some(sorting1), 500); + let row_group2 = + create_row_group_with_sorting(&schema_descr, Some(sorting2), 500); + + let metadata = + create_parquet_metadata(schema_descr, vec![row_group1, row_group2]); + + let result = + DFParquetMetadata::ordering_from_parquet_metadata(&metadata, &arrow_schema) + .unwrap(); + + assert!(result.is_none()); + } + + #[test] + fn test_ordering_from_parquet_metadata_no_row_groups() { + // No row groups → should return None + let schema_descr = create_test_schema_descr(&["a", "b"]); + let arrow_schema = create_test_arrow_schema(&["a", "b"]); + + let metadata = create_parquet_metadata(schema_descr, vec![]); + + let result = + DFParquetMetadata::ordering_from_parquet_metadata(&metadata, &arrow_schema) + .unwrap(); + + assert!(result.is_none()); + } + + #[test] + fn test_ordering_from_parquet_metadata_no_sorting_columns() { + // Row groups with no sorting_columns → should return None + let schema_descr = create_test_schema_descr(&["a", "b"]); + let arrow_schema = create_test_arrow_schema(&["a", "b"]); + + let row_group = create_row_group_with_sorting(&schema_descr, None, 1000); + let metadata = create_parquet_metadata(schema_descr, vec![row_group]); + + let result = + DFParquetMetadata::ordering_from_parquet_metadata(&metadata, &arrow_schema) + .unwrap(); + + assert!(result.is_none()); + } + + #[test] + fn test_ordering_from_parquet_metadata_empty_sorting_columns() { + // Row groups with empty sorting_columns vector → should return None + let schema_descr = create_test_schema_descr(&["a", "b"]); + let arrow_schema = create_test_arrow_schema(&["a", "b"]); + + let row_group = create_row_group_with_sorting(&schema_descr, Some(vec![]), 1000); + let metadata = create_parquet_metadata(schema_descr, vec![row_group]); + + let result = + DFParquetMetadata::ordering_from_parquet_metadata(&metadata, &arrow_schema) + .unwrap(); + + assert!(result.is_none()); + } + + #[test] + fn test_ordering_from_parquet_metadata_column_index_out_of_bounds() { + // Sorting column index references a non-existent column → should return None + let schema_descr = create_test_schema_descr(&["a", "b"]); + let arrow_schema = create_test_arrow_schema(&["a", "b"]); + + // Column index 5 doesn't exist (only 0 and 1 are valid) + let sorting_columns = vec![SortingColumn { + column_idx: 5, + descending: false, + nulls_first: true, + }]; + + let row_group = + create_row_group_with_sorting(&schema_descr, Some(sorting_columns), 1000); + let metadata = create_parquet_metadata(schema_descr, vec![row_group]); + + let result = + DFParquetMetadata::ordering_from_parquet_metadata(&metadata, &arrow_schema) + .unwrap(); + + assert!(result.is_none()); + } + + #[test] + fn test_ordering_from_parquet_metadata_column_not_in_arrow_schema() { + // Parquet has more columns than Arrow schema + // If sorting references a column not in Arrow schema, should return None + let schema_descr = create_test_schema_descr(&["a", "b", "c"]); + // Arrow schema only has "a" and "b", not "c" + let arrow_schema = create_test_arrow_schema(&["a", "b"]); + + // Sort by column "c" (index 2 in Parquet) + let sorting_columns = vec![SortingColumn { + column_idx: 2, + descending: false, + nulls_first: true, + }]; + + let row_group = + create_row_group_with_sorting(&schema_descr, Some(sorting_columns), 1000); + let metadata = create_parquet_metadata(schema_descr, vec![row_group]); + + let result = + DFParquetMetadata::ordering_from_parquet_metadata(&metadata, &arrow_schema) + .unwrap(); + + // Column "c" is not in Arrow schema, so ordering should be None + assert!(result.is_none()); + } + + #[test] + fn test_ordering_from_parquet_metadata_first_has_sorting_second_has_none() { + // First row group has sorting, second has None → should return None + let schema_descr = create_test_schema_descr(&["a", "b"]); + let arrow_schema = create_test_arrow_schema(&["a", "b"]); + + let sorting = vec![SortingColumn { + column_idx: 0, + descending: false, + nulls_first: true, + }]; + + let row_group1 = create_row_group_with_sorting(&schema_descr, Some(sorting), 500); + let row_group2 = create_row_group_with_sorting(&schema_descr, None, 500); + + let metadata = + create_parquet_metadata(schema_descr, vec![row_group1, row_group2]); + + let result = + DFParquetMetadata::ordering_from_parquet_metadata(&metadata, &arrow_schema) + .unwrap(); + + assert!(result.is_none()); + } + #[test] fn test_has_any_exact_match() { // Case 1: Mixed exact and inexact matches @@ -666,4 +1209,101 @@ mod tests { assert_eq!(result, Some(false)); } } + + #[test] + fn test_sort_expr_to_sorting_column_asc_nulls_first() { + use super::sort_expr_to_sorting_column; + + let column = Arc::new(Column::new("a", 0)); + let sort_expr = PhysicalSortExpr::new( + column, + SortOptions { + descending: false, + nulls_first: true, + }, + ); + + let result = sort_expr_to_sorting_column(&sort_expr).unwrap(); + + assert_eq!(result.column_idx, 0); + assert!(!result.descending); + assert!(result.nulls_first); + } + + #[test] + fn test_sort_expr_to_sorting_column_desc_nulls_last() { + use super::sort_expr_to_sorting_column; + + let column = Arc::new(Column::new("b", 5)); + let sort_expr = PhysicalSortExpr::new( + column, + SortOptions { + descending: true, + nulls_first: false, + }, + ); + + let result = sort_expr_to_sorting_column(&sort_expr).unwrap(); + + assert_eq!(result.column_idx, 5); + assert!(result.descending); + assert!(!result.nulls_first); + } + + #[test] + fn test_sort_expr_to_sorting_column_non_column_expr() { + use super::sort_expr_to_sorting_column; + use datafusion_common::ScalarValue; + use datafusion_physical_expr::expressions::Literal; + + // Create a non-column expression (a literal) + let literal = Arc::new(Literal::new(ScalarValue::Int32(Some(42)))); + let sort_expr = PhysicalSortExpr::new(literal, SortOptions::default()); + + let result = sort_expr_to_sorting_column(&sort_expr); + + assert!(result.is_err()); + let err = result.unwrap_err(); + assert!( + err.to_string() + .contains("only supports simple column references"), + "Expected error about column references, got: {err}" + ); + } + + #[test] + fn test_lex_ordering_to_sorting_columns() { + use super::lex_ordering_to_sorting_columns; + + let col_a = Arc::new(Column::new("a", 0)); + let col_b = Arc::new(Column::new("b", 1)); + + let ordering = LexOrdering::new(vec![ + PhysicalSortExpr::new( + col_a, + SortOptions { + descending: false, + nulls_first: true, + }, + ), + PhysicalSortExpr::new( + col_b, + SortOptions { + descending: true, + nulls_first: false, + }, + ), + ]) + .unwrap(); + + let result = lex_ordering_to_sorting_columns(&ordering).unwrap(); + + assert_eq!(result.len(), 2); + assert_eq!(result[0].column_idx, 0); + assert!(!result[0].descending); + assert!(result[0].nulls_first); + assert_eq!(result[1].column_idx, 1); + assert!(result[1].descending); + assert!(!result[1].nulls_first); + } } diff --git a/datafusion/datasource-parquet/src/row_group_filter.rs b/datafusion/datasource-parquet/src/row_group_filter.rs index 1264197609f3f..09c110b818b32 100644 --- a/datafusion/datasource-parquet/src/row_group_filter.rs +++ b/datafusion/datasource-parquet/src/row_group_filter.rs @@ -1565,6 +1565,7 @@ mod tests { range: None, statistics: None, extensions: None, + ordering: None, metadata_size_hint: None, }; diff --git a/datafusion/datasource/src/display.rs b/datafusion/datasource/src/display.rs index 15fe8679acdaf..1503b57c5bea3 100644 --- a/datafusion/datasource/src/display.rs +++ b/datafusion/datasource/src/display.rs @@ -293,6 +293,7 @@ mod tests { range: None, statistics: None, extensions: None, + ordering: None, metadata_size_hint: None, } } diff --git a/datafusion/datasource/src/file_format.rs b/datafusion/datasource/src/file_format.rs index 54389ecd214e5..8372f775efdf9 100644 --- a/datafusion/datasource/src/file_format.rs +++ b/datafusion/datasource/src/file_format.rs @@ -32,6 +32,7 @@ use arrow::datatypes::SchemaRef; use datafusion_common::file_options::file_type::FileType; use datafusion_common::{GetExt, Result, Statistics, internal_err, not_impl_err}; use datafusion_physical_expr::LexRequirement; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_plan::ExecutionPlan; use datafusion_session::Session; @@ -41,6 +42,40 @@ use object_store::{ObjectMeta, ObjectStore}; /// Default max records to scan to infer the schema pub const DEFAULT_SCHEMA_INFER_MAX_RECORD: usize = 1000; +/// Metadata fetched from a file, including statistics and ordering. +/// +/// This struct is returned by [`FileFormat::infer_stats_and_ordering`] to +/// provide all metadata in a single read, avoiding duplicate I/O operations. +/// +/// Note: Individual components (statistics and ordering) are typically cached +/// separately by `FileStatisticsCache` implementations to enable partial cache +/// hits. For example, statistics may be cached from a previous query while +/// ordering is fetched fresh. +#[derive(Debug, Clone)] +#[non_exhaustive] +pub struct FileMeta { + /// Statistics for the file (row counts, byte sizes, column statistics). + pub statistics: Statistics, + /// The ordering (sort order) of the file, if known. + pub ordering: Option, +} + +impl FileMeta { + /// Creates a new `FileMeta` with the given statistics and no ordering. + pub fn new(statistics: Statistics) -> Self { + Self { + statistics, + ordering: None, + } + } + + /// Sets the ordering for this file metadata. + pub fn with_ordering(mut self, ordering: Option) -> Self { + self.ordering = ordering; + self + } +} + /// This trait abstracts all the file format specific implementations /// from the [`TableProvider`]. This helps code re-utilization across /// providers that support the same file formats. @@ -90,6 +125,52 @@ pub trait FileFormat: Send + Sync + fmt::Debug { object: &ObjectMeta, ) -> Result; + /// Infer the ordering (sort order) for the provided object from file metadata. + /// + /// Returns `Ok(None)` if the file format does not support ordering inference + /// or if the file does not have ordering information. + /// + /// `table_schema` is the (combined) schema of the overall table + /// and may be a superset of the schema contained in this file. + /// + /// The default implementation returns `Ok(None)`. + async fn infer_ordering( + &self, + _state: &dyn Session, + _store: &Arc, + _table_schema: SchemaRef, + _object: &ObjectMeta, + ) -> Result> { + Ok(None) + } + + /// Infer both statistics and ordering from a single metadata read. + /// + /// This is more efficient than calling [`Self::infer_stats`] and + /// [`Self::infer_ordering`] separately when both are needed, as it avoids + /// reading file metadata twice. + /// + /// The default implementation calls both methods separately. File formats + /// that can extract both from a single read should override this method. + async fn infer_stats_and_ordering( + &self, + state: &dyn Session, + store: &Arc, + table_schema: SchemaRef, + object: &ObjectMeta, + ) -> Result { + let statistics = self + .infer_stats(state, store, Arc::clone(&table_schema), object) + .await?; + let ordering = self + .infer_ordering(state, store, table_schema, object) + .await?; + Ok(FileMeta { + statistics, + ordering, + }) + } + /// Take a list of files and convert it to the appropriate executor /// according to this file format. async fn create_physical_plan( diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index c8636343ccc5a..3395bc05887f8 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -392,7 +392,34 @@ impl FileScanConfigBuilder { } /// Set the output ordering of the files + /// + /// If files have orderings defined (e.g., from Parquet metadata), this method + /// validates that the provided output_ordering is compatible with the file orderings. + /// A warning is logged if there's a mismatch, but the provided ordering is still used. pub fn with_output_ordering(mut self, output_ordering: Vec) -> Self { + // Check if files have orderings defined. + // If the files have no ordering, use the provided output_ordering. + // If they do have ordering, validate compatibility and log a warning if mismatched. + if !output_ordering.is_empty() { + for group in self.file_groups.iter() { + for file in group.iter() { + if let Some(file_ordering) = &file.ordering { + // Check if any of the output orderings is compatible with file ordering + let compatible = output_ordering + .iter() + .any(|oo| is_ordering_compatible(oo, file_ordering)); + if !compatible { + log::warn!( + "Output ordering {:?} may not be compatible with file ordering {:?} for file {}", + output_ordering, + file_ordering, + file.object_meta.location + ); + } + } + } + } + } self.output_ordering = output_ordering; self } @@ -1337,6 +1364,37 @@ pub fn wrap_partition_value_in_dict(val: ScalarValue) -> ScalarValue { ScalarValue::Dictionary(Box::new(DataType::UInt16), Box::new(val)) } +/// Checks if a user-specified output ordering is compatible with a file's ordering. +/// +/// An output ordering is considered compatible with a file ordering if: +/// - The output ordering is a prefix of the file ordering, or +/// - The output ordering equals the file ordering +/// +/// For example, if a file has ordering `[a ASC, b ASC, c ASC]`: +/// - `[a ASC]` is compatible (prefix) +/// - `[a ASC, b ASC]` is compatible (prefix) +/// - `[a ASC, b ASC, c ASC]` is compatible (equal) +/// - `[a ASC, b DESC]` is NOT compatible (different sort direction) +/// - `[b ASC]` is NOT compatible (not a prefix) +fn is_ordering_compatible( + output_ordering: &LexOrdering, + file_ordering: &LexOrdering, +) -> bool { + // Output ordering must not be longer than file ordering + if output_ordering.len() > file_ordering.len() { + return false; + } + + // Check that each element of output_ordering matches the corresponding element in file_ordering + for (output_expr, file_expr) in output_ordering.iter().zip(file_ordering.iter()) { + if output_expr != file_expr { + return false; + } + } + + true +} + #[cfg(test)] mod tests { use std::collections::HashMap; @@ -1695,6 +1753,7 @@ mod tests { }) .collect::>(), })), + ordering: None, extensions: None, metadata_size_hint: None, } @@ -2306,4 +2365,123 @@ mod tests { _ => panic!("Expected Hash partitioning"), } } + + // Tests for is_ordering_compatible + // Note: LexOrdering is non-degenerate (always has at least one element), + // so we don't need to test empty orderings. + mod ordering_compatibility_tests { + use super::*; + use arrow::compute::SortOptions; + + /// Helper to create a PhysicalSortExpr + fn sort_expr( + name: &str, + idx: usize, + descending: bool, + nulls_first: bool, + ) -> PhysicalSortExpr { + PhysicalSortExpr::new( + Arc::new(Column::new(name, idx)), + SortOptions { + descending, + nulls_first, + }, + ) + } + + /// Helper to create a non-empty LexOrdering (unwraps the Option) + fn lex_ordering(exprs: Vec) -> LexOrdering { + LexOrdering::new(exprs).expect("expected non-empty ordering") + } + + #[test] + fn test_is_ordering_compatible_equal_orderings() { + // Equal orderings should be compatible + let output = lex_ordering(vec![ + sort_expr("a", 0, false, true), + sort_expr("b", 1, true, false), + ]); + let file = lex_ordering(vec![ + sort_expr("a", 0, false, true), + sort_expr("b", 1, true, false), + ]); + + assert!(is_ordering_compatible(&output, &file)); + } + + #[test] + fn test_is_ordering_compatible_output_is_prefix() { + // Output ordering is prefix of file ordering → compatible + let output = lex_ordering(vec![sort_expr("a", 0, false, true)]); + let file = lex_ordering(vec![ + sort_expr("a", 0, false, true), + sort_expr("b", 1, true, false), + ]); + + assert!(is_ordering_compatible(&output, &file)); + } + + #[test] + fn test_is_ordering_compatible_output_longer() { + // Output ordering longer than file ordering → not compatible + let output = lex_ordering(vec![ + sort_expr("a", 0, false, true), + sort_expr("b", 1, true, false), + sort_expr("c", 2, false, true), + ]); + let file = lex_ordering(vec![ + sort_expr("a", 0, false, true), + sort_expr("b", 1, true, false), + ]); + + assert!(!is_ordering_compatible(&output, &file)); + } + + #[test] + fn test_is_ordering_compatible_different_direction() { + // Different sort direction → not compatible + let output = lex_ordering(vec![ + sort_expr("a", 0, false, true), // ASC + ]); + let file = lex_ordering(vec![ + sort_expr("a", 0, true, true), // DESC + ]); + + assert!(!is_ordering_compatible(&output, &file)); + } + + #[test] + fn test_is_ordering_compatible_different_nulls() { + // Different nulls_first → not compatible + let output = lex_ordering(vec![ + sort_expr("a", 0, false, true), // NULLS FIRST + ]); + let file = lex_ordering(vec![ + sort_expr("a", 0, false, false), // NULLS LAST + ]); + + assert!(!is_ordering_compatible(&output, &file)); + } + + #[test] + fn test_is_ordering_compatible_different_columns() { + // Different columns → not compatible + let output = lex_ordering(vec![sort_expr("a", 0, false, true)]); + let file = lex_ordering(vec![sort_expr("b", 1, false, true)]); + + assert!(!is_ordering_compatible(&output, &file)); + } + + #[test] + fn test_is_ordering_compatible_different_column_indices() { + // Same column name but different indices → not compatible + // (Column expressions with same name but different indices are not equal) + let output = lex_ordering(vec![sort_expr("a", 0, false, true)]); + let file = lex_ordering(vec![ + sort_expr("a", 1, false, true), // Different index + ]); + + assert!(!is_ordering_compatible(&output, &file)); + } + } } diff --git a/datafusion/datasource/src/mod.rs b/datafusion/datasource/src/mod.rs index 347e783c278d0..b78593a0ddcf7 100644 --- a/datafusion/datasource/src/mod.rs +++ b/datafusion/datasource/src/mod.rs @@ -58,6 +58,7 @@ use chrono::TimeZone; use datafusion_common::stats::Precision; use datafusion_common::{ColumnStatistics, Result, exec_datafusion_err}; use datafusion_common::{ScalarValue, Statistics}; +use datafusion_physical_expr::LexOrdering; use futures::{Stream, StreamExt}; use object_store::{GetOptions, GetRange, ObjectStore}; use object_store::{ObjectMeta, path::Path}; @@ -133,6 +134,8 @@ pub struct PartitionedFile { /// When set via [`Self::with_statistics`], partition column statistics are automatically /// computed from [`Self::partition_values`] with exact min/max/null_count/distinct_count. pub statistics: Option>, + /// A known ordering of the data in this file. + pub ordering: Option, /// An optional field for user defined per object metadata pub extensions: Option>, /// The estimated size of the parquet metadata, in bytes @@ -153,6 +156,20 @@ impl PartitionedFile { partition_values: vec![], range: None, statistics: None, + ordering: None, + extensions: None, + metadata_size_hint: None, + } + } + + /// Create a file from a known ObjectMeta without partition + pub fn new_from_meta(object_meta: ObjectMeta) -> Self { + Self { + object_meta, + partition_values: vec![], + range: None, + statistics: None, + ordering: None, extensions: None, metadata_size_hint: None, } @@ -170,6 +187,7 @@ impl PartitionedFile { }, partition_values: vec![], range: Some(FileRange { start, end }), + ordering: None, statistics: None, extensions: None, metadata_size_hint: None, @@ -177,6 +195,13 @@ impl PartitionedFile { .with_range(start, end) } + /// Attach partition values to this file. + /// This replaces any existing partition values. + pub fn with_partition_values(mut self, partition_values: Vec) -> Self { + self.partition_values = partition_values; + self + } + /// Size of the file to be scanned (taking into account the range, if present). pub fn effective_size(&self) -> u64 { if let Some(range) = &self.range { @@ -282,6 +307,15 @@ impl PartitionedFile { false } } + + /// Set the known ordering of data in this file. + /// + /// The ordering represents the lexicographical sort order of the data, + /// typically inferred from file metadata (e.g., Parquet sorting_columns). + pub fn with_ordering(mut self, ordering: Option) -> Self { + self.ordering = ordering; + self + } } impl From for PartitionedFile { @@ -292,6 +326,7 @@ impl From for PartitionedFile { range: None, statistics: None, extensions: None, + ordering: None, metadata_size_hint: None, } } @@ -483,6 +518,7 @@ pub fn generate_test_files(num_files: usize, overlap_factor: f64) -> Vec( .map(|res| res.map_err(|e| DataFusionError::ObjectStore(Box::new(e)))) .boxed()), Some(cache) => { - // Convert prefix to Option for cache lookup - let prefix_filter = prefix.cloned(); - - // Try cache lookup with optional prefix filter - let vec = if let Some(res) = - cache.get_with_extra(table_base_path, &prefix_filter) - { + // Try cache lookup + let vec = if let Some(cached) = cache.get(table_base_path) { debug!("Hit list files cache"); - res.as_ref().clone() + // Cache hit - apply prefix filter if needed + if prefix.is_some() { + let full_prefix_str = full_prefix.as_ref(); + cached + .files + .iter() + .filter(|meta| { + meta.location.as_ref().starts_with(full_prefix_str) + }) + .cloned() + .collect() + } else { + cached.files.as_ref().clone() + } } else { // Cache miss - always list and cache the full table // This ensures we have complete data for future prefix queries @@ -380,7 +387,7 @@ async fn list_with_cache<'b>( .list(Some(table_base_path)) .try_collect::>() .await?; - cache.put(table_base_path, Arc::new(vec.clone())); + cache.put(table_base_path, CachedFileList::new(vec.clone())); // If a prefix filter was requested, apply it to the results if prefix.is_some() { @@ -494,6 +501,7 @@ mod tests { use std::any::Any; use std::collections::HashMap; use std::ops::Range; + use std::sync::Arc; use tempfile::tempdir; #[test] diff --git a/datafusion/execution/Cargo.toml b/datafusion/execution/Cargo.toml index ca1fba07cae2d..e36352a15e94b 100644 --- a/datafusion/execution/Cargo.toml +++ b/datafusion/execution/Cargo.toml @@ -55,6 +55,7 @@ chrono = { workspace = true } dashmap = { workspace = true } datafusion-common = { workspace = true, default-features = false } datafusion-expr = { workspace = true, default-features = false } +datafusion-physical-expr-common = { workspace = true, default-features = false } futures = { workspace = true } log = { workspace = true } object_store = { workspace = true, features = ["fs"] } @@ -66,3 +67,4 @@ url = { workspace = true } [dev-dependencies] insta = { workspace = true } +datafusion-physical-expr = { workspace = true, default-features = false } diff --git a/datafusion/execution/src/cache/cache_manager.rs b/datafusion/execution/src/cache/cache_manager.rs index c76a68c651eb0..46d5bcb4e6142 100644 --- a/datafusion/execution/src/cache/cache_manager.rs +++ b/datafusion/execution/src/cache/cache_manager.rs @@ -15,10 +15,11 @@ // specific language governing permissions and limitations // under the License. +use crate::cache::DefaultListFilesCache; use crate::cache::cache_unit::DefaultFilesMetadataCache; -use crate::cache::{CacheAccessor, DefaultListFilesCache}; use datafusion_common::stats::Precision; use datafusion_common::{Result, Statistics}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use object_store::ObjectMeta; use object_store::path::Path; use std::any::Any; @@ -31,16 +32,92 @@ pub use super::list_files_cache::{ DEFAULT_LIST_FILES_CACHE_MEMORY_LIMIT, DEFAULT_LIST_FILES_CACHE_TTL, }; -/// A cache for [`Statistics`]. +/// Cached metadata for a file, including statistics and ordering. +/// +/// This struct embeds the [`ObjectMeta`] used for cache validation, +/// along with the cached statistics and ordering information. +#[derive(Debug, Clone)] +pub struct CachedFileMetadata { + /// File metadata used for cache validation (size, last_modified). + pub meta: ObjectMeta, + /// Cached statistics for the file, if available. + pub statistics: Arc, + /// Cached ordering for the file. + pub ordering: Option, +} + +impl CachedFileMetadata { + /// Create a new cached file metadata entry. + pub fn new( + meta: ObjectMeta, + statistics: Arc, + ordering: Option, + ) -> Self { + Self { + meta, + statistics, + ordering, + } + } + + /// Check if this cached entry is still valid for the given metadata. + /// + /// Returns true if the file size and last modified time match. + pub fn is_valid_for(&self, current_meta: &ObjectMeta) -> bool { + self.meta.size == current_meta.size + && self.meta.last_modified == current_meta.last_modified + } +} + +/// A cache for file statistics and orderings. +/// +/// This cache stores [`CachedFileMetadata`] which includes: +/// - File metadata for validation (size, last_modified) +/// - Statistics for the file +/// - Ordering information for the file /// /// If enabled via [`CacheManagerConfig::with_files_statistics_cache`] this /// cache avoids inferring the same file statistics repeatedly during the /// session lifetime. /// +/// The typical usage pattern is: +/// 1. Call `get(path)` to check for cached value +/// 2. If `Some(cached)`, validate with `cached.is_valid_for(¤t_meta)` +/// 3. If invalid or missing, compute new value and call `put(path, new_value)` +/// /// See [`crate::runtime_env::RuntimeEnv`] for more details -pub trait FileStatisticsCache: - CacheAccessor, Extra = ObjectMeta> -{ +pub trait FileStatisticsCache: Send + Sync { + /// Get a cached entry if it exists. + /// + /// Returns the cached value without any validation. The caller should + /// validate the returned value using `cached.is_valid_for(¤t_meta)`. + fn get(&self, key: &Path) -> Option; + + /// Store a value in the cache. + /// + /// Returns the previous value if one existed. + fn put(&self, key: &Path, value: CachedFileMetadata) -> Option; + + /// Remove an entry from the cache, returning the value if it existed. + fn remove(&self, k: &Path) -> Option; + + /// Check if the cache contains a specific key. + fn contains_key(&self, k: &Path) -> bool; + + /// Fetch the total number of cache entries. + fn len(&self) -> usize; + + /// Check if the cache collection is empty. + fn is_empty(&self) -> bool { + self.len() == 0 + } + + /// Remove all entries from the cache. + fn clear(&self); + + /// Return the cache name. + fn name(&self) -> String; + /// Retrieves the information about the entries currently cached. fn list_entries(&self) -> HashMap; } @@ -58,6 +135,39 @@ pub struct FileStatisticsCacheEntry { pub table_size_bytes: Precision, /// Size of the statistics entry, in bytes. pub statistics_size_bytes: usize, + /// Whether ordering information is cached for this file. + pub has_ordering: bool, +} + +/// Cached file listing. +/// +/// TTL expiration is handled internally by the cache implementation. +#[derive(Debug, Clone)] +pub struct CachedFileList { + /// The cached file list. + pub files: Arc>, +} + +impl CachedFileList { + /// Create a new cached file list. + pub fn new(files: Vec) -> Self { + Self { + files: Arc::new(files), + } + } + + /// Filter the files by prefix. + pub fn filter_by_prefix(&self, prefix: &Option) -> Vec { + match prefix { + Some(prefix) => self + .files + .iter() + .filter(|meta| meta.location.as_ref().starts_with(prefix.as_ref())) + .cloned() + .collect(), + None => self.files.as_ref().clone(), + } + } } /// Cache for storing the [`ObjectMeta`]s that result from listing a path @@ -67,21 +177,43 @@ pub struct FileStatisticsCacheEntry { /// especially when done over remote object stores. /// /// The cache key is always the table's base path, ensuring a stable cache key. -/// The `Extra` type is `Option`, representing an optional prefix filter -/// (relative to the table base path) for partition-aware lookups. -/// -/// When `get_with_extra(key, Some(prefix))` is called: -/// - The cache entry for `key` (table base path) is fetched -/// - Results are filtered to only include files matching `key/prefix` -/// - Filtered results are returned without making a storage call +/// The cached value is a [`CachedFileList`] containing the files and a timestamp. /// -/// This enables efficient partition pruning: a single cached listing of the -/// full table can serve queries for any partition subset. +/// Partition filtering is done after retrieval using [`CachedFileList::filter_by_prefix`]. /// /// See [`crate::runtime_env::RuntimeEnv`] for more details. -pub trait ListFilesCache: - CacheAccessor>, Extra = Option> -{ +pub trait ListFilesCache: Send + Sync { + /// Get a cached entry if it exists. + /// + /// Returns the cached file list. TTL validation is handled internally + /// by the implementation - expired entries return `None`. + fn get(&self, key: &Path) -> Option; + + /// Store a value in the cache. + /// + /// Returns the previous value if one existed. + fn put(&self, key: &Path, value: CachedFileList) -> Option; + + /// Remove an entry from the cache, returning the value if it existed. + fn remove(&self, k: &Path) -> Option; + + /// Check if the cache contains a specific key. + fn contains_key(&self, k: &Path) -> bool; + + /// Fetch the total number of cache entries. + fn len(&self) -> usize; + + /// Check if the cache collection is empty. + fn is_empty(&self) -> bool { + self.len() == 0 + } + + /// Remove all entries from the cache. + fn clear(&self); + + /// Return the cache name. + fn name(&self) -> String; + /// Returns the cache's memory limit in bytes. fn cache_limit(&self) -> usize; @@ -113,9 +245,44 @@ pub trait FileMetadata: Any + Send + Sync { fn extra_info(&self) -> HashMap; } +/// Cached file metadata entry with validation information. +#[derive(Clone)] +pub struct CachedFileMetadataEntry { + /// File metadata used for cache validation (size, last_modified). + pub meta: ObjectMeta, + /// The cached file metadata. + pub file_metadata: Arc, +} + +impl CachedFileMetadataEntry { + /// Create a new cached file metadata entry. + pub fn new(meta: ObjectMeta, file_metadata: Arc) -> Self { + Self { + meta, + file_metadata, + } + } + + /// Check if this cached entry is still valid for the given metadata. + pub fn is_valid_for(&self, current_meta: &ObjectMeta) -> bool { + self.meta.size == current_meta.size + && self.meta.last_modified == current_meta.last_modified + } +} + +impl Debug for CachedFileMetadataEntry { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + f.debug_struct("CachedFileMetadataEntry") + .field("meta", &self.meta) + .field("memory_size", &self.file_metadata.memory_size()) + .finish() + } +} + /// Cache for file-embedded metadata. /// -/// This cache stores per-file metadata in the form of [`FileMetadata`], +/// This cache stores per-file metadata in the form of [`CachedFileMetadataEntry`], +/// which includes the [`ObjectMeta`] for validation. /// /// For example, the built in [`ListingTable`] uses this cache to avoid parsing /// Parquet footers multiple times for the same file. @@ -124,12 +291,50 @@ pub trait FileMetadata: Any + Send + Sync { /// and users can also provide their own implementations to implement custom /// caching strategies. /// +/// The typical usage pattern is: +/// 1. Call `get(path)` to check for cached value +/// 2. If `Some(cached)`, validate with `cached.is_valid_for(¤t_meta)` +/// 3. If invalid or missing, compute new value and call `put(path, new_value)` +/// /// See [`crate::runtime_env::RuntimeEnv`] for more details. /// /// [`ListingTable`]: https://docs.rs/datafusion/latest/datafusion/datasource/listing/struct.ListingTable.html -pub trait FileMetadataCache: - CacheAccessor, Extra = ObjectMeta> -{ +pub trait FileMetadataCache: Send + Sync { + /// Get a cached entry if it exists. + /// + /// Returns the cached value without any validation. The caller should + /// validate the returned value using `cached.is_valid_for(¤t_meta)`. + fn get(&self, key: &Path) -> Option; + + /// Store a value in the cache. + /// + /// Returns the previous value if one existed. + fn put( + &self, + key: &Path, + value: CachedFileMetadataEntry, + ) -> Option; + + /// Remove an entry from the cache, returning the value if it existed. + fn remove(&self, k: &Path) -> Option; + + /// Check if the cache contains a specific key. + fn contains_key(&self, k: &Path) -> bool; + + /// Fetch the total number of cache entries. + fn len(&self) -> usize; + + /// Check if the cache collection is empty. + fn is_empty(&self) -> bool { + self.len() == 0 + } + + /// Remove all entries from the cache. + fn clear(&self); + + /// Return the cache name. + fn name(&self) -> String; + /// Returns the cache's memory limit in bytes. fn cache_limit(&self) -> usize; diff --git a/datafusion/execution/src/cache/cache_unit.rs b/datafusion/execution/src/cache/cache_unit.rs index 5351df449a7c1..29c56e4d902f1 100644 --- a/datafusion/execution/src/cache/cache_unit.rs +++ b/datafusion/execution/src/cache/cache_unit.rs @@ -16,189 +16,287 @@ // under the License. use std::collections::HashMap; -use std::sync::Arc; -use crate::cache::CacheAccessor; -use crate::cache::cache_manager::{FileStatisticsCache, FileStatisticsCacheEntry}; - -use datafusion_common::Statistics; +use crate::cache::cache_manager::{ + CachedFileMetadata, FileStatisticsCache, FileStatisticsCacheEntry, +}; use dashmap::DashMap; -use object_store::ObjectMeta; use object_store::path::Path; pub use crate::cache::DefaultFilesMetadataCache; /// Default implementation of [`FileStatisticsCache`] /// -/// Stores collected statistics for files +/// Stores cached file metadata (statistics and orderings) for files. +/// +/// The typical usage pattern is: +/// 1. Call `get(path)` to check for cached value +/// 2. If `Some(cached)`, validate with `cached.is_valid_for(¤t_meta)` +/// 3. If invalid or missing, compute new value and call `put(path, new_value)` /// -/// Cache is invalided when file size or last modification has changed +/// Uses DashMap for lock-free concurrent access. /// /// [`FileStatisticsCache`]: crate::cache::cache_manager::FileStatisticsCache #[derive(Default)] pub struct DefaultFileStatisticsCache { - statistics: DashMap)>, + cache: DashMap, } impl FileStatisticsCache for DefaultFileStatisticsCache { - fn list_entries(&self) -> HashMap { - let mut entries = HashMap::::new(); - - for entry in &self.statistics { - let path = entry.key(); - let (object_meta, stats) = entry.value(); - entries.insert( - path.clone(), - FileStatisticsCacheEntry { - object_meta: object_meta.clone(), - num_rows: stats.num_rows, - num_columns: stats.column_statistics.len(), - table_size_bytes: stats.total_byte_size, - statistics_size_bytes: 0, // TODO: set to the real size in the future - }, - ); - } - - entries - } -} - -impl CacheAccessor> for DefaultFileStatisticsCache { - type Extra = ObjectMeta; - - /// Get `Statistics` for file location. - fn get(&self, k: &Path) -> Option> { - self.statistics - .get(k) - .map(|s| Some(Arc::clone(&s.value().1))) - .unwrap_or(None) - } - - /// Get `Statistics` for file location. Returns None if file has changed or not found. - fn get_with_extra(&self, k: &Path, e: &Self::Extra) -> Option> { - self.statistics - .get(k) - .map(|s| { - let (saved_meta, statistics) = s.value(); - if saved_meta.size != e.size - || saved_meta.last_modified != e.last_modified - { - // file has changed - None - } else { - Some(Arc::clone(statistics)) - } - }) - .unwrap_or(None) - } - - /// Save collected file statistics - fn put(&self, _key: &Path, _value: Arc) -> Option> { - panic!("Put cache in DefaultFileStatisticsCache without Extra not supported.") + fn get(&self, key: &Path) -> Option { + self.cache.get(key).map(|entry| entry.value().clone()) } - fn put_with_extra( - &self, - key: &Path, - value: Arc, - e: &Self::Extra, - ) -> Option> { - self.statistics - .insert(key.clone(), (e.clone(), value)) - .map(|x| x.1) + fn put(&self, key: &Path, value: CachedFileMetadata) -> Option { + self.cache.insert(key.clone(), value) } - fn remove(&self, k: &Path) -> Option> { - self.statistics.remove(k).map(|x| x.1.1) + fn remove(&self, k: &Path) -> Option { + self.cache.remove(k).map(|(_, entry)| entry) } fn contains_key(&self, k: &Path) -> bool { - self.statistics.contains_key(k) + self.cache.contains_key(k) } fn len(&self) -> usize { - self.statistics.len() + self.cache.len() } fn clear(&self) { - self.statistics.clear() + self.cache.clear(); } + fn name(&self) -> String { "DefaultFileStatisticsCache".to_string() } + + fn list_entries(&self) -> HashMap { + let mut entries = HashMap::::new(); + + for entry in self.cache.iter() { + let path = entry.key(); + let cached = entry.value(); + entries.insert( + path.clone(), + FileStatisticsCacheEntry { + object_meta: cached.meta.clone(), + num_rows: cached.statistics.num_rows, + num_columns: cached.statistics.column_statistics.len(), + table_size_bytes: cached.statistics.total_byte_size, + statistics_size_bytes: 0, // TODO: set to the real size in the future + has_ordering: cached.ordering.is_some(), + }, + ); + } + + entries + } } #[cfg(test)] mod tests { use super::*; - use crate::cache::CacheAccessor; - use crate::cache::cache_manager::{FileStatisticsCache, FileStatisticsCacheEntry}; - use crate::cache::cache_unit::DefaultFileStatisticsCache; + use crate::cache::cache_manager::{ + CachedFileMetadata, FileStatisticsCache, FileStatisticsCacheEntry, + }; use arrow::datatypes::{DataType, Field, Schema, TimeUnit}; use chrono::DateTime; use datafusion_common::Statistics; use datafusion_common::stats::Precision; + use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; use object_store::ObjectMeta; use object_store::path::Path; + use std::sync::Arc; - #[test] - fn test_statistics_cache() { - let meta = ObjectMeta { - location: Path::from("test"), + fn create_test_meta(path: &str, size: u64) -> ObjectMeta { + ObjectMeta { + location: Path::from(path), last_modified: DateTime::parse_from_rfc3339("2022-09-27T22:36:00+02:00") .unwrap() .into(), - size: 1024, + size, e_tag: None, version: None, - }; + } + } + + #[test] + fn test_statistics_cache() { + let meta = create_test_meta("test", 1024); let cache = DefaultFileStatisticsCache::default(); - assert!(cache.get_with_extra(&meta.location, &meta).is_none()); - - cache.put_with_extra( - &meta.location, - Statistics::new_unknown(&Schema::new(vec![Field::new( - "test_column", - DataType::Timestamp(TimeUnit::Second, None), - false, - )])) - .into(), - &meta, + + let schema = Schema::new(vec![Field::new( + "test_column", + DataType::Timestamp(TimeUnit::Second, None), + false, + )]); + + // Cache miss + assert!(cache.get(&meta.location).is_none()); + + // Put a value + let cached_value = CachedFileMetadata::new( + meta.clone(), + Arc::new(Statistics::new_unknown(&schema)), + None, + ); + cache.put(&meta.location, cached_value); + + // Cache hit + let result = cache.get(&meta.location); + assert!(result.is_some()); + let cached = result.unwrap(); + assert!(cached.is_valid_for(&meta)); + + // File size changed - validation should fail + let meta2 = create_test_meta("test", 2048); + let cached = cache.get(&meta2.location).unwrap(); + assert!(!cached.is_valid_for(&meta2)); + + // Update with new value + let cached_value2 = CachedFileMetadata::new( + meta2.clone(), + Arc::new(Statistics::new_unknown(&schema)), + None, + ); + cache.put(&meta2.location, cached_value2); + + // Test list_entries + let entries = cache.list_entries(); + assert_eq!(entries.len(), 1); + let entry = entries.get(&Path::from("test")).unwrap(); + assert_eq!(entry.object_meta.size, 2048); // Should be updated value + } + + fn ordering(schema: &Schema) -> LexOrdering { + let expr = datafusion_physical_expr::expressions::col( + schema.fields().iter().next().unwrap().name(), + schema, + ) + .unwrap(); + LexOrdering::new(vec![PhysicalSortExpr::new_default(expr)]).unwrap() + } + + #[test] + fn test_ordering_cache() { + let meta = create_test_meta("test.parquet", 100); + let cache = DefaultFileStatisticsCache::default(); + + let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]); + + // Cache statistics with no ordering + let cached_value = CachedFileMetadata::new( + meta.clone(), + Arc::new(Statistics::new_unknown(&schema)), + None, // No ordering yet + ); + cache.put(&meta.location, cached_value); + + let result = cache.get(&meta.location).unwrap(); + assert!(result.ordering.is_none()); + + // Update to add ordering + let mut cached = cache.get(&meta.location).unwrap(); + if cached.is_valid_for(&meta) && cached.ordering.is_none() { + cached.ordering = Some(ordering(&schema)); + } + cache.put(&meta.location, cached); + + let result2 = cache.get(&meta.location).unwrap(); + assert!(result2.ordering.is_some()); + + // Verify list_entries shows has_ordering = true + let entries = cache.list_entries(); + assert_eq!(entries.len(), 1); + assert!(entries.get(&meta.location).unwrap().has_ordering); + } + + #[test] + fn test_cache_invalidation_on_file_modification() { + let cache = DefaultFileStatisticsCache::default(); + let path = Path::from("test.parquet"); + let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]); + + let meta_v1 = create_test_meta("test.parquet", 100); + + // Cache initial value + let cached_value = CachedFileMetadata::new( + meta_v1.clone(), + Arc::new(Statistics::new_unknown(&schema)), + None, + ); + cache.put(&path, cached_value); + + // File modified (size changed) + let meta_v2 = create_test_meta("test.parquet", 200); + + let cached = cache.get(&path).unwrap(); + // Should not be valid for new meta + assert!(!cached.is_valid_for(&meta_v2)); + + // Compute new value and update + let new_cached = CachedFileMetadata::new( + meta_v2.clone(), + Arc::new(Statistics::new_unknown(&schema)), + None, ); - assert!(cache.get_with_extra(&meta.location, &meta).is_some()); - - // file size changed - let mut meta2 = meta.clone(); - meta2.size = 2048; - assert!(cache.get_with_extra(&meta2.location, &meta2).is_none()); - - // file last_modified changed - let mut meta2 = meta.clone(); - meta2.last_modified = DateTime::parse_from_rfc3339("2022-09-27T22:40:00+02:00") - .unwrap() - .into(); - assert!(cache.get_with_extra(&meta2.location, &meta2).is_none()); - - // different file - let mut meta2 = meta.clone(); - meta2.location = Path::from("test2"); - assert!(cache.get_with_extra(&meta2.location, &meta2).is_none()); - - // test the list_entries method + cache.put(&path, new_cached); + + // Should have new metadata + let result = cache.get(&path).unwrap(); + assert_eq!(result.meta.size, 200); + } + + #[test] + fn test_list_entries() { + let cache = DefaultFileStatisticsCache::default(); + let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]); + + let meta1 = create_test_meta("test1.parquet", 100); + + let cached_value = CachedFileMetadata::new( + meta1.clone(), + Arc::new(Statistics::new_unknown(&schema)), + None, + ); + cache.put(&meta1.location, cached_value); + let meta2 = create_test_meta("test2.parquet", 200); + let cached_value = CachedFileMetadata::new( + meta2.clone(), + Arc::new(Statistics::new_unknown(&schema)), + Some(ordering(&schema)), + ); + cache.put(&meta2.location, cached_value); + let entries = cache.list_entries(); assert_eq!( entries, - HashMap::from([( - Path::from("test"), - FileStatisticsCacheEntry { - object_meta: meta.clone(), - num_rows: Precision::Absent, - num_columns: 1, - table_size_bytes: Precision::Absent, - statistics_size_bytes: 0, - } - )]) + HashMap::from([ + ( + Path::from("test1.parquet"), + FileStatisticsCacheEntry { + object_meta: meta1, + num_rows: Precision::Absent, + num_columns: 1, + table_size_bytes: Precision::Absent, + statistics_size_bytes: 0, + has_ordering: false, + } + ), + ( + Path::from("test2.parquet"), + FileStatisticsCacheEntry { + object_meta: meta2, + num_rows: Precision::Absent, + num_columns: 1, + table_size_bytes: Precision::Absent, + statistics_size_bytes: 0, + has_ordering: true, + } + ), + ]) ); } } diff --git a/datafusion/execution/src/cache/file_metadata_cache.rs b/datafusion/execution/src/cache/file_metadata_cache.rs index c7a24dd878e4f..b22a46bd3d621 100644 --- a/datafusion/execution/src/cache/file_metadata_cache.rs +++ b/datafusion/execution/src/cache/file_metadata_cache.rs @@ -15,22 +15,18 @@ // specific language governing permissions and limitations // under the License. -use std::{ - collections::HashMap, - sync::{Arc, Mutex}, -}; +use std::{collections::HashMap, sync::Mutex}; -use object_store::{ObjectMeta, path::Path}; +use object_store::path::Path; use crate::cache::{ - CacheAccessor, - cache_manager::{FileMetadata, FileMetadataCache, FileMetadataCacheEntry}, + cache_manager::{CachedFileMetadataEntry, FileMetadataCache, FileMetadataCacheEntry}, lru_queue::LruQueue, }; /// Handles the inner state of the [`DefaultFilesMetadataCache`] struct. struct DefaultFilesMetadataCacheState { - lru_queue: LruQueue)>, + lru_queue: LruQueue, memory_limit: usize, memory_used: usize, cache_hits: HashMap, @@ -46,35 +42,18 @@ impl DefaultFilesMetadataCacheState { } } - /// Returns the respective entry from the cache, if it exists and the `size` and `last_modified` - /// properties from [`ObjectMeta`] match. + /// Returns the respective entry from the cache, if it exists. /// If the entry exists, it becomes the most recently used. - fn get(&mut self, k: &ObjectMeta) -> Option> { - self.lru_queue - .get(&k.location) - .map(|(object_meta, metadata)| { - if object_meta.size != k.size - || object_meta.last_modified != k.last_modified - { - None - } else { - *self.cache_hits.entry(k.location.clone()).or_insert(0) += 1; - Some(Arc::clone(metadata)) - } - }) - .unwrap_or(None) + fn get(&mut self, k: &Path) -> Option { + self.lru_queue.get(k).cloned().inspect(|_| { + *self.cache_hits.entry(k.clone()).or_insert(0) += 1; + }) } - /// Checks if the metadata is currently cached (entry exists and the `size` and `last_modified` - /// properties of [`ObjectMeta`] match). + /// Checks if the metadata is currently cached. /// The LRU queue is not updated. - fn contains_key(&self, k: &ObjectMeta) -> bool { - self.lru_queue - .peek(&k.location) - .map(|(object_meta, _)| { - object_meta.size == k.size && object_meta.last_modified == k.last_modified - }) - .unwrap_or(false) + fn contains_key(&self, k: &Path) -> bool { + self.lru_queue.peek(k).is_some() } /// Adds a new key-value pair to cache, meaning LRU entries might be evicted if required. @@ -82,35 +61,34 @@ impl DefaultFilesMetadataCacheState { /// If the size of the metadata is greater than the `memory_limit`, the value is not inserted. fn put( &mut self, - key: ObjectMeta, - value: Arc, - ) -> Option> { - let value_size = value.memory_size(); + key: Path, + value: CachedFileMetadataEntry, + ) -> Option { + let value_size = value.file_metadata.memory_size(); // no point in trying to add this value to the cache if it cannot fit entirely if value_size > self.memory_limit { return None; } - self.cache_hits.insert(key.location.clone(), 0); + self.cache_hits.insert(key.clone(), 0); // if the key is already in the cache, the old value is removed - let old_value = self.lru_queue.put(key.location.clone(), (key, value)); + let old_value = self.lru_queue.put(key, value); self.memory_used += value_size; - if let Some((_, ref old_metadata)) = old_value { - self.memory_used -= old_metadata.memory_size(); + if let Some(ref old_entry) = old_value { + self.memory_used -= old_entry.file_metadata.memory_size(); } self.evict_entries(); - old_value.map(|v| v.1) + old_value } /// Evicts entries from the LRU cache until `memory_used` is lower than `memory_limit`. fn evict_entries(&mut self) { while self.memory_used > self.memory_limit { if let Some(removed) = self.lru_queue.pop() { - let metadata: Arc = removed.1.1; - self.memory_used -= metadata.memory_size(); + self.memory_used -= removed.1.file_metadata.memory_size(); } else { // cache is empty while memory_used > memory_limit, cannot happen debug_assert!( @@ -123,11 +101,11 @@ impl DefaultFilesMetadataCacheState { } /// Removes an entry from the cache and returns it, if it exists. - fn remove(&mut self, k: &ObjectMeta) -> Option> { - if let Some((_, old_metadata)) = self.lru_queue.remove(&k.location) { - self.memory_used -= old_metadata.memory_size(); - self.cache_hits.remove(&k.location); - Some(old_metadata) + fn remove(&mut self, k: &Path) -> Option { + if let Some(old_entry) = self.lru_queue.remove(k) { + self.memory_used -= old_entry.file_metadata.memory_size(); + self.cache_hits.remove(k); + Some(old_entry) } else { None } @@ -150,8 +128,8 @@ impl DefaultFilesMetadataCacheState { /// /// Collected file embedded metadata cache. /// -/// The metadata for each file is invalidated when the file size or last -/// modification time have been changed. +/// The metadata for each file is validated by comparing the cached [`ObjectMeta`] +/// (size and last_modified) against the current file state using `cached.is_valid_for(¤t_meta)`. /// /// # Internal details /// @@ -160,11 +138,7 @@ impl DefaultFilesMetadataCacheState { /// size of the cached entries exceeds `memory_limit`, the least recently used entries /// are evicted until the total size is lower than `memory_limit`. /// -/// # `Extra` Handling -/// -/// Users should use the [`Self::get`] and [`Self::put`] methods. The -/// [`Self::get_with_extra`] and [`Self::put_with_extra`] methods simply call -/// `get` and `put`, respectively. +/// [`ObjectMeta`]: object_store::ObjectMeta pub struct DefaultFilesMetadataCache { // the state is wrapped in a Mutex to ensure the operations are atomic state: Mutex, @@ -190,77 +164,26 @@ impl DefaultFilesMetadataCache { } impl FileMetadataCache for DefaultFilesMetadataCache { - fn cache_limit(&self) -> usize { - let state = self.state.lock().unwrap(); - state.memory_limit - } - - fn update_cache_limit(&self, limit: usize) { - let mut state = self.state.lock().unwrap(); - state.memory_limit = limit; - state.evict_entries(); - } - - fn list_entries(&self) -> HashMap { - let state = self.state.lock().unwrap(); - let mut entries = HashMap::::new(); - - for (path, (object_meta, metadata)) in state.lru_queue.list_entries() { - entries.insert( - path.clone(), - FileMetadataCacheEntry { - object_meta: object_meta.clone(), - size_bytes: metadata.memory_size(), - hits: *state.cache_hits.get(path).expect("entry must exist"), - extra: metadata.extra_info(), - }, - ); - } - - entries - } -} - -impl CacheAccessor> for DefaultFilesMetadataCache { - type Extra = ObjectMeta; - - fn get(&self, k: &ObjectMeta) -> Option> { + fn get(&self, key: &Path) -> Option { let mut state = self.state.lock().unwrap(); - state.get(k) - } - - fn get_with_extra( - &self, - k: &ObjectMeta, - _e: &Self::Extra, - ) -> Option> { - self.get(k) + state.get(key) } fn put( &self, - key: &ObjectMeta, - value: Arc, - ) -> Option> { + key: &Path, + value: CachedFileMetadataEntry, + ) -> Option { let mut state = self.state.lock().unwrap(); state.put(key.clone(), value) } - fn put_with_extra( - &self, - key: &ObjectMeta, - value: Arc, - _e: &Self::Extra, - ) -> Option> { - self.put(key, value) - } - - fn remove(&self, k: &ObjectMeta) -> Option> { + fn remove(&self, k: &Path) -> Option { let mut state = self.state.lock().unwrap(); state.remove(k) } - fn contains_key(&self, k: &ObjectMeta) -> bool { + fn contains_key(&self, k: &Path) -> bool { let state = self.state.lock().unwrap(); state.contains_key(k) } @@ -278,6 +201,36 @@ impl CacheAccessor> for DefaultFilesMetadataCa fn name(&self) -> String { "DefaultFilesMetadataCache".to_string() } + + fn cache_limit(&self) -> usize { + let state = self.state.lock().unwrap(); + state.memory_limit + } + + fn update_cache_limit(&self, limit: usize) { + let mut state = self.state.lock().unwrap(); + state.memory_limit = limit; + state.evict_entries(); + } + + fn list_entries(&self) -> HashMap { + let state = self.state.lock().unwrap(); + let mut entries = HashMap::::new(); + + for (path, entry) in state.lru_queue.list_entries() { + entries.insert( + path.clone(), + FileMetadataCacheEntry { + object_meta: entry.meta.clone(), + size_bytes: entry.file_metadata.memory_size(), + hits: *state.cache_hits.get(path).expect("entry must exist"), + extra: entry.file_metadata.extra_info(), + }, + ); + } + + entries + } } #[cfg(test)] @@ -285,9 +238,8 @@ mod tests { use std::collections::HashMap; use std::sync::Arc; - use crate::cache::CacheAccessor; use crate::cache::cache_manager::{ - FileMetadata, FileMetadataCache, FileMetadataCacheEntry, + CachedFileMetadataEntry, FileMetadata, FileMetadataCache, FileMetadataCacheEntry, }; use crate::cache::file_metadata_cache::DefaultFilesMetadataCache; use object_store::ObjectMeta; @@ -311,67 +263,77 @@ mod tests { } } - #[test] - fn test_default_file_metadata_cache() { - let object_meta = ObjectMeta { - location: Path::from("test"), + fn create_test_object_meta(path: &str, size: usize) -> ObjectMeta { + ObjectMeta { + location: Path::from(path), last_modified: chrono::DateTime::parse_from_rfc3339( "2025-07-29T12:12:12+00:00", ) .unwrap() .into(), - size: 1024, + size: size as u64, e_tag: None, version: None, - }; + } + } + + #[test] + fn test_default_file_metadata_cache() { + let object_meta = create_test_object_meta("test", 1024); let metadata: Arc = Arc::new(TestFileMetadata { metadata: "retrieved_metadata".to_owned(), }); let cache = DefaultFilesMetadataCache::new(1024 * 1024); - assert!(cache.get(&object_meta).is_none()); - // put - cache.put(&object_meta, Arc::clone(&metadata)); + // Cache miss + assert!(cache.get(&object_meta.location).is_none()); + + // Put a value + let cached_entry = + CachedFileMetadataEntry::new(object_meta.clone(), Arc::clone(&metadata)); + cache.put(&object_meta.location, cached_entry); - // get and contains of a valid entry - assert!(cache.contains_key(&object_meta)); - let value = cache.get(&object_meta); - assert!(value.is_some()); - let test_file_metadata = Arc::downcast::(value.unwrap()); + // Verify the cached value + assert!(cache.contains_key(&object_meta.location)); + let result = cache.get(&object_meta.location).unwrap(); + let test_file_metadata = Arc::downcast::(result.file_metadata); assert!(test_file_metadata.is_ok()); assert_eq!(test_file_metadata.unwrap().metadata, "retrieved_metadata"); - // file size changed - let mut object_meta2 = object_meta.clone(); - object_meta2.size = 2048; - assert!(cache.get(&object_meta2).is_none()); - assert!(!cache.contains_key(&object_meta2)); - - // file last_modified changed - let mut object_meta2 = object_meta.clone(); - object_meta2.last_modified = - chrono::DateTime::parse_from_rfc3339("2025-07-29T13:13:13+00:00") - .unwrap() - .into(); - assert!(cache.get(&object_meta2).is_none()); - assert!(!cache.contains_key(&object_meta2)); - - // different file - let mut object_meta2 = object_meta.clone(); - object_meta2.location = Path::from("test2"); - assert!(cache.get(&object_meta2).is_none()); - assert!(!cache.contains_key(&object_meta2)); + // Cache hit - check validation + let result2 = cache.get(&object_meta.location).unwrap(); + assert!(result2.is_valid_for(&object_meta)); + + // File size changed - closure should detect invalidity + let object_meta2 = create_test_object_meta("test", 2048); + let result3 = cache.get(&object_meta2.location).unwrap(); + // Cached entry should NOT be valid for new meta + assert!(!result3.is_valid_for(&object_meta2)); + + // Return new entry + let new_entry = + CachedFileMetadataEntry::new(object_meta2.clone(), Arc::clone(&metadata)); + cache.put(&object_meta2.location, new_entry); + + let result4 = cache.get(&object_meta2.location).unwrap(); + assert_eq!(result4.meta.size, 2048); // remove - cache.remove(&object_meta); - assert!(cache.get(&object_meta).is_none()); - assert!(!cache.contains_key(&object_meta)); + cache.remove(&object_meta.location); + assert!(!cache.contains_key(&object_meta.location)); // len and clear - cache.put(&object_meta, Arc::clone(&metadata)); - cache.put(&object_meta2, metadata); + let object_meta3 = create_test_object_meta("test3", 100); + cache.put( + &object_meta.location, + CachedFileMetadataEntry::new(object_meta.clone(), Arc::clone(&metadata)), + ); + cache.put( + &object_meta3.location, + CachedFileMetadataEntry::new(object_meta3.clone(), Arc::clone(&metadata)), + ); assert_eq!(cache.len(), 2); cache.clear(); assert_eq!(cache.len(), 0); @@ -402,92 +364,129 @@ mod tests { let (object_meta2, metadata2) = generate_test_metadata_with_size("2", 500); let (object_meta3, metadata3) = generate_test_metadata_with_size("3", 300); - cache.put(&object_meta1, metadata1); - cache.put(&object_meta2, metadata2); - cache.put(&object_meta3, metadata3); + cache.put( + &object_meta1.location, + CachedFileMetadataEntry::new(object_meta1.clone(), metadata1), + ); + cache.put( + &object_meta2.location, + CachedFileMetadataEntry::new(object_meta2.clone(), metadata2), + ); + cache.put( + &object_meta3.location, + CachedFileMetadataEntry::new(object_meta3.clone(), metadata3), + ); // all entries will fit assert_eq!(cache.len(), 3); assert_eq!(cache.memory_used(), 900); - assert!(cache.contains_key(&object_meta1)); - assert!(cache.contains_key(&object_meta2)); - assert!(cache.contains_key(&object_meta3)); + assert!(cache.contains_key(&object_meta1.location)); + assert!(cache.contains_key(&object_meta2.location)); + assert!(cache.contains_key(&object_meta3.location)); // add a new entry which will remove the least recently used ("1") let (object_meta4, metadata4) = generate_test_metadata_with_size("4", 200); - cache.put(&object_meta4, metadata4); + cache.put( + &object_meta4.location, + CachedFileMetadataEntry::new(object_meta4.clone(), metadata4), + ); assert_eq!(cache.len(), 3); assert_eq!(cache.memory_used(), 1000); - assert!(!cache.contains_key(&object_meta1)); - assert!(cache.contains_key(&object_meta4)); + assert!(!cache.contains_key(&object_meta1.location)); + assert!(cache.contains_key(&object_meta4.location)); // get entry "2", which will move it to the top of the queue, and add a new one which will // remove the new least recently used ("3") - cache.get(&object_meta2); + let _ = cache.get(&object_meta2.location); let (object_meta5, metadata5) = generate_test_metadata_with_size("5", 100); - cache.put(&object_meta5, metadata5); + cache.put( + &object_meta5.location, + CachedFileMetadataEntry::new(object_meta5.clone(), metadata5), + ); assert_eq!(cache.len(), 3); assert_eq!(cache.memory_used(), 800); - assert!(!cache.contains_key(&object_meta3)); - assert!(cache.contains_key(&object_meta5)); + assert!(!cache.contains_key(&object_meta3.location)); + assert!(cache.contains_key(&object_meta5.location)); // new entry which will not be able to fit in the 1000 bytes allocated let (object_meta6, metadata6) = generate_test_metadata_with_size("6", 1200); - cache.put(&object_meta6, metadata6); + cache.put( + &object_meta6.location, + CachedFileMetadataEntry::new(object_meta6.clone(), metadata6), + ); assert_eq!(cache.len(), 3); assert_eq!(cache.memory_used(), 800); - assert!(!cache.contains_key(&object_meta6)); + assert!(!cache.contains_key(&object_meta6.location)); // new entry which is able to fit without removing any entry let (object_meta7, metadata7) = generate_test_metadata_with_size("7", 200); - cache.put(&object_meta7, metadata7); + cache.put( + &object_meta7.location, + CachedFileMetadataEntry::new(object_meta7.clone(), metadata7), + ); assert_eq!(cache.len(), 4); assert_eq!(cache.memory_used(), 1000); - assert!(cache.contains_key(&object_meta7)); + assert!(cache.contains_key(&object_meta7.location)); // new entry which will remove all other entries let (object_meta8, metadata8) = generate_test_metadata_with_size("8", 999); - cache.put(&object_meta8, metadata8); + cache.put( + &object_meta8.location, + CachedFileMetadataEntry::new(object_meta8.clone(), metadata8), + ); assert_eq!(cache.len(), 1); assert_eq!(cache.memory_used(), 999); - assert!(cache.contains_key(&object_meta8)); + assert!(cache.contains_key(&object_meta8.location)); // when updating an entry, the previous ones are not unnecessarily removed let (object_meta9, metadata9) = generate_test_metadata_with_size("9", 300); let (object_meta10, metadata10) = generate_test_metadata_with_size("10", 200); let (object_meta11_v1, metadata11_v1) = generate_test_metadata_with_size("11", 400); - cache.put(&object_meta9, metadata9); - cache.put(&object_meta10, metadata10); - cache.put(&object_meta11_v1, metadata11_v1); + cache.put( + &object_meta9.location, + CachedFileMetadataEntry::new(object_meta9.clone(), metadata9), + ); + cache.put( + &object_meta10.location, + CachedFileMetadataEntry::new(object_meta10.clone(), metadata10), + ); + cache.put( + &object_meta11_v1.location, + CachedFileMetadataEntry::new(object_meta11_v1.clone(), metadata11_v1), + ); assert_eq!(cache.memory_used(), 900); assert_eq!(cache.len(), 3); let (object_meta11_v2, metadata11_v2) = generate_test_metadata_with_size("11", 500); - cache.put(&object_meta11_v2, metadata11_v2); + cache.put( + &object_meta11_v2.location, + CachedFileMetadataEntry::new(object_meta11_v2.clone(), metadata11_v2), + ); assert_eq!(cache.memory_used(), 1000); assert_eq!(cache.len(), 3); - assert!(cache.contains_key(&object_meta9)); - assert!(cache.contains_key(&object_meta10)); - assert!(cache.contains_key(&object_meta11_v2)); - assert!(!cache.contains_key(&object_meta11_v1)); + assert!(cache.contains_key(&object_meta9.location)); + assert!(cache.contains_key(&object_meta10.location)); + assert!(cache.contains_key(&object_meta11_v2.location)); // when updating an entry that now exceeds the limit, the LRU ("9") needs to be removed let (object_meta11_v3, metadata11_v3) = generate_test_metadata_with_size("11", 501); - cache.put(&object_meta11_v3, metadata11_v3); + cache.put( + &object_meta11_v3.location, + CachedFileMetadataEntry::new(object_meta11_v3.clone(), metadata11_v3), + ); assert_eq!(cache.memory_used(), 701); assert_eq!(cache.len(), 2); - assert!(cache.contains_key(&object_meta10)); - assert!(cache.contains_key(&object_meta11_v3)); - assert!(!cache.contains_key(&object_meta11_v2)); + assert!(cache.contains_key(&object_meta10.location)); + assert!(cache.contains_key(&object_meta11_v3.location)); // manually removing an entry that is not the LRU - cache.remove(&object_meta11_v3); + cache.remove(&object_meta11_v3.location); assert_eq!(cache.len(), 1); assert_eq!(cache.memory_used(), 200); - assert!(cache.contains_key(&object_meta10)); - assert!(!cache.contains_key(&object_meta11_v3)); + assert!(cache.contains_key(&object_meta10.location)); + assert!(!cache.contains_key(&object_meta11_v3.location)); // clear cache.clear(); @@ -498,17 +497,26 @@ mod tests { let (object_meta12, metadata12) = generate_test_metadata_with_size("12", 300); let (object_meta13, metadata13) = generate_test_metadata_with_size("13", 200); let (object_meta14, metadata14) = generate_test_metadata_with_size("14", 500); - cache.put(&object_meta12, metadata12); - cache.put(&object_meta13, metadata13); - cache.put(&object_meta14, metadata14); + cache.put( + &object_meta12.location, + CachedFileMetadataEntry::new(object_meta12.clone(), metadata12), + ); + cache.put( + &object_meta13.location, + CachedFileMetadataEntry::new(object_meta13.clone(), metadata13), + ); + cache.put( + &object_meta14.location, + CachedFileMetadataEntry::new(object_meta14.clone(), metadata14), + ); assert_eq!(cache.len(), 3); assert_eq!(cache.memory_used(), 1000); cache.update_cache_limit(600); assert_eq!(cache.len(), 1); assert_eq!(cache.memory_used(), 500); - assert!(!cache.contains_key(&object_meta12)); - assert!(!cache.contains_key(&object_meta13)); - assert!(cache.contains_key(&object_meta14)); + assert!(!cache.contains_key(&object_meta12.location)); + assert!(!cache.contains_key(&object_meta13.location)); + assert!(cache.contains_key(&object_meta14.location)); } #[test] @@ -519,9 +527,18 @@ mod tests { let (object_meta3, metadata3) = generate_test_metadata_with_size("3", 300); // initial entries, all will have hits = 0 - cache.put(&object_meta1, metadata1); - cache.put(&object_meta2, metadata2); - cache.put(&object_meta3, metadata3); + cache.put( + &object_meta1.location, + CachedFileMetadataEntry::new(object_meta1.clone(), metadata1), + ); + cache.put( + &object_meta2.location, + CachedFileMetadataEntry::new(object_meta2.clone(), metadata2), + ); + cache.put( + &object_meta3.location, + CachedFileMetadataEntry::new(object_meta3.clone(), metadata3), + ); assert_eq!( cache.list_entries(), HashMap::from([ @@ -565,7 +582,7 @@ mod tests { ); // new hit on "1" - cache.get(&object_meta1); + let _ = cache.get(&object_meta1.location); assert_eq!( cache.list_entries(), HashMap::from([ @@ -610,7 +627,10 @@ mod tests { // new entry, will evict "2" let (object_meta4, metadata4) = generate_test_metadata_with_size("4", 600); - cache.put(&object_meta4, metadata4); + cache.put( + &object_meta4.location, + CachedFileMetadataEntry::new(object_meta4.clone(), metadata4), + ); assert_eq!( cache.list_entries(), HashMap::from([ @@ -655,7 +675,10 @@ mod tests { // replace entry "1" let (object_meta1_new, metadata1_new) = generate_test_metadata_with_size("1", 50); - cache.put(&object_meta1_new, metadata1_new); + cache.put( + &object_meta1_new.location, + CachedFileMetadataEntry::new(object_meta1_new.clone(), metadata1_new), + ); assert_eq!( cache.list_entries(), HashMap::from([ @@ -699,7 +722,7 @@ mod tests { ); // remove entry "4" - cache.remove(&object_meta4); + cache.remove(&object_meta4.location); assert_eq!( cache.list_entries(), HashMap::from([ diff --git a/datafusion/execution/src/cache/list_files_cache.rs b/datafusion/execution/src/cache/list_files_cache.rs index 661bc47b5468a..1e92fe2600f69 100644 --- a/datafusion/execution/src/cache/list_files_cache.rs +++ b/datafusion/execution/src/cache/list_files_cache.rs @@ -24,7 +24,10 @@ use std::{ use datafusion_common::instant::Instant; use object_store::{ObjectMeta, path::Path}; -use crate::cache::{CacheAccessor, cache_manager::ListFilesCache, lru_queue::LruQueue}; +use crate::cache::{ + cache_manager::{CachedFileList, ListFilesCache}, + lru_queue::LruQueue, +}; pub trait TimeProvider: Send + Sync + 'static { fn now(&self) -> Instant; @@ -50,11 +53,10 @@ impl TimeProvider for SystemTimeProvider { /// the cache exceeds `memory_limit`, the least recently used entries are evicted until the total /// size is lower than the `memory_limit`. /// -/// # `Extra` Handling +/// # Cache API /// -/// Users should use the [`Self::get`] and [`Self::put`] methods. The -/// [`Self::get_with_extra`] and [`Self::put_with_extra`] methods simply call -/// `get` and `put`, respectively. +/// Uses `get` and `put` methods for cache operations. TTL validation is handled internally - +/// expired entries return `None` from `get`. pub struct DefaultListFilesCache { state: Mutex, time_provider: Arc, @@ -84,42 +86,29 @@ impl DefaultListFilesCache { self.time_provider = provider; self } - - /// Returns the cache's memory limit in bytes. - pub fn cache_limit(&self) -> usize { - self.state.lock().unwrap().memory_limit - } - - /// Updates the cache with a new memory limit in bytes. - pub fn update_cache_limit(&self, limit: usize) { - let mut state = self.state.lock().unwrap(); - state.memory_limit = limit; - state.evict_entries(); - } - - /// Returns the TTL (time-to-live) applied to cache entries. - pub fn cache_ttl(&self) -> Option { - self.state.lock().unwrap().ttl - } } struct ListFilesEntry { - metas: Arc>, + cached_file_list: CachedFileList, size_bytes: usize, expires: Option, } impl ListFilesEntry { fn try_new( - metas: Arc>, + cached_file_list: CachedFileList, ttl: Option, now: Instant, ) -> Option { - let size_bytes = (metas.capacity() * size_of::()) - + metas.iter().map(meta_heap_bytes).reduce(|acc, b| acc + b)?; + let size_bytes = (cached_file_list.files.capacity() * size_of::()) + + cached_file_list + .files + .iter() + .map(meta_heap_bytes) + .reduce(|acc, b| acc + b)?; Some(Self { - metas, + cached_file_list, size_bytes, expires: ttl.map(|t| now + t), }) @@ -175,65 +164,22 @@ impl DefaultListFilesCacheState { } } - /// Performs a prefix-aware cache lookup. - /// - /// # Arguments - /// * `table_base` - The table's base path (the cache key) - /// * `prefix` - Optional prefix filter relative to the table base path - /// * `now` - Current time for expiration checking - /// - /// # Behavior - /// - Fetches the cache entry for `table_base` - /// - If `prefix` is `Some`, filters results to only files matching `table_base/prefix` - /// - Returns the (potentially filtered) results + /// Gets an entry from the cache, checking for expiration. /// - /// # Example - /// ```text - /// get_with_prefix("my_table", Some("a=1"), now) - /// → Fetch cache entry for "my_table" - /// → Filter to files matching "my_table/a=1/*" - /// → Return filtered results - /// ``` - fn get_with_prefix( - &mut self, - table_base: &Path, - prefix: Option<&Path>, - now: Instant, - ) -> Option>> { - let entry = self.lru_queue.get(table_base)?; + /// Returns the cached file list if it exists and hasn't expired. + /// If the entry has expired, it is removed from the cache. + fn get(&mut self, key: &Path, now: Instant) -> Option { + let entry = self.lru_queue.get(key)?; // Check expiration if let Some(exp) = entry.expires && now > exp { - self.remove(table_base); + self.remove(key); return None; } - // Early return if no prefix filter - return all files - let Some(prefix) = prefix else { - return Some(Arc::clone(&entry.metas)); - }; - - // Build the full prefix path: table_base/prefix - let mut parts: Vec<_> = table_base.parts().collect(); - parts.extend(prefix.parts()); - let full_prefix = Path::from_iter(parts); - let full_prefix_str = full_prefix.as_ref(); - - // Filter files to only those matching the prefix - let filtered: Vec = entry - .metas - .iter() - .filter(|meta| meta.location.as_ref().starts_with(full_prefix_str)) - .cloned() - .collect(); - - if filtered.is_empty() { - None - } else { - Some(Arc::new(filtered)) - } + Some(entry.cached_file_list.clone()) } /// Checks if the respective entry is currently cached. @@ -263,9 +209,9 @@ impl DefaultListFilesCacheState { fn put( &mut self, key: &Path, - value: Arc>, + value: CachedFileList, now: Instant, - ) -> Option>> { + ) -> Option { let entry = ListFilesEntry::try_new(value, self.ttl, now)?; let entry_size = entry.size_bytes; @@ -284,7 +230,7 @@ impl DefaultListFilesCacheState { self.evict_entries(); - old_value.map(|v| v.metas) + old_value.map(|v| v.cached_file_list) } /// Evicts entries from the LRU cache until `memory_used` is lower than `memory_limit`. @@ -304,10 +250,10 @@ impl DefaultListFilesCacheState { } /// Removes an entry from the cache and returns it, if it exists. - fn remove(&mut self, k: &Path) -> Option>> { + fn remove(&mut self, k: &Path) -> Option { if let Some(entry) = self.lru_queue.remove(k) { self.memory_used -= entry.size_bytes; - Some(entry.metas) + Some(entry.cached_file_list) } else { None } @@ -326,82 +272,19 @@ impl DefaultListFilesCacheState { } impl ListFilesCache for DefaultListFilesCache { - fn cache_limit(&self) -> usize { - let state = self.state.lock().unwrap(); - state.memory_limit - } - - fn cache_ttl(&self) -> Option { - let state = self.state.lock().unwrap(); - state.ttl - } - - fn update_cache_limit(&self, limit: usize) { - let mut state = self.state.lock().unwrap(); - state.memory_limit = limit; - state.evict_entries(); - } - - fn update_cache_ttl(&self, ttl: Option) { - let mut state = self.state.lock().unwrap(); - state.ttl = ttl; - state.evict_entries(); - } -} - -impl CacheAccessor>> for DefaultListFilesCache { - type Extra = Option; - - /// Gets all files for the given table base path. - /// - /// This is equivalent to calling `get_with_extra(k, &None)`. - fn get(&self, k: &Path) -> Option>> { - self.get_with_extra(k, &None) - } - - /// Performs a prefix-aware cache lookup. - /// - /// # Arguments - /// * `table_base` - The table's base path (the cache key) - /// * `prefix` - Optional prefix filter (relative to table base) for partition filtering - /// - /// # Behavior - /// - Fetches the cache entry for `table_base` - /// - If `prefix` is `Some`, filters results to only files matching `table_base/prefix` - /// - Returns the (potentially filtered) results - /// - /// This enables efficient partition pruning - a single cached listing of the full table - /// can serve queries for any partition subset without additional storage calls. - fn get_with_extra( - &self, - table_base: &Path, - prefix: &Self::Extra, - ) -> Option>> { + fn get(&self, key: &Path) -> Option { let mut state = self.state.lock().unwrap(); let now = self.time_provider.now(); - state.get_with_prefix(table_base, prefix.as_ref(), now) + state.get(key, now) } - fn put( - &self, - key: &Path, - value: Arc>, - ) -> Option>> { + fn put(&self, key: &Path, value: CachedFileList) -> Option { let mut state = self.state.lock().unwrap(); let now = self.time_provider.now(); state.put(key, value, now) } - fn put_with_extra( - &self, - key: &Path, - value: Arc>, - _e: &Self::Extra, - ) -> Option>> { - self.put(key, value) - } - - fn remove(&self, k: &Path) -> Option>> { + fn remove(&self, k: &Path) -> Option { let mut state = self.state.lock().unwrap(); state.remove(k) } @@ -425,6 +308,28 @@ impl CacheAccessor>> for DefaultListFilesCache { fn name(&self) -> String { String::from("DefaultListFilesCache") } + + fn cache_limit(&self) -> usize { + let state = self.state.lock().unwrap(); + state.memory_limit + } + + fn cache_ttl(&self) -> Option { + let state = self.state.lock().unwrap(); + state.ttl + } + + fn update_cache_limit(&self, limit: usize) { + let mut state = self.state.lock().unwrap(); + state.memory_limit = limit; + state.evict_entries(); + } + + fn update_cache_ttl(&self, ttl: Option) { + let mut state = self.state.lock().unwrap(); + state.ttl = ttl; + state.evict_entries(); + } } #[cfg(test)] @@ -478,22 +383,21 @@ mod tests { } } - /// Helper function to create a vector of ObjectMeta with at least meta_size bytes + /// Helper function to create a CachedFileList with at least meta_size bytes fn create_test_list_files_entry( path: &str, count: usize, meta_size: usize, - ) -> (Path, Arc>, usize) { + ) -> (Path, CachedFileList, usize) { let metas: Vec = (0..count) .map(|i| create_test_object_meta(&format!("file{i}"), meta_size)) .collect(); - let metas = Arc::new(metas); // Calculate actual size using the same logic as ListFilesEntry::try_new let size = (metas.capacity() * size_of::()) + metas.iter().map(meta_heap_bytes).sum::(); - (Path::from(path), metas, size) + (Path::from(path), CachedFileList::new(metas), size) } #[test] @@ -502,25 +406,25 @@ mod tests { let path = Path::from("test_path"); // Initially cache is empty - assert!(cache.get(&path).is_none()); assert!(!cache.contains_key(&path)); assert_eq!(cache.len(), 0); - // Put an entry + // Cache miss - get returns None + assert!(cache.get(&path).is_none()); + + // Put a value let meta = create_test_object_meta("file1", 50); - let value = Arc::new(vec![meta.clone()]); - cache.put(&path, Arc::clone(&value)); + cache.put(&path, CachedFileList::new(vec![meta])); - // Entry should be retrievable + // Entry should be cached assert!(cache.contains_key(&path)); assert_eq!(cache.len(), 1); - let retrieved = cache.get(&path).unwrap(); - assert_eq!(retrieved.len(), 1); - assert_eq!(retrieved[0].location, meta.location); + let result = cache.get(&path).unwrap(); + assert_eq!(result.files.len(), 1); // Remove the entry let removed = cache.remove(&path).unwrap(); - assert_eq!(removed.len(), 1); + assert_eq!(removed.files.len(), 1); assert!(!cache.contains_key(&path)); assert_eq!(cache.len(), 0); @@ -583,7 +487,7 @@ mod tests { // Access path1 to move it to front (MRU) // Order is now: path2 (LRU), path3, path1 (MRU) - cache.get(&path1); + let _ = cache.get(&path1); // Adding a new entry should evict path2 (the LRU) let (path4, value4, _) = create_test_list_files_entry("path4", 1, 100); @@ -609,6 +513,7 @@ mod tests { assert_eq!(cache.len(), 2); // Try to add an entry that's too large to fit in the cache + // The entry is not stored (too large) let (path_large, value_large, _) = create_test_list_files_entry("large", 1, 1000); cache.put(&path_large, value_large); @@ -681,7 +586,7 @@ mod tests { // Add three entries cache.put(&path1, value1); cache.put(&path2, value2); - cache.put(&path3, value3_v1); + cache.put(&path3.clone(), value3_v1); assert_eq!(cache.len(), 3); // Update path3 with same size - should not cause eviction @@ -715,8 +620,6 @@ mod tests { cache.put(&path2, value2); // Entries should be accessible immediately - assert!(cache.get(&path1).is_some()); - assert!(cache.get(&path2).is_some()); assert!(cache.contains_key(&path1)); assert!(cache.contains_key(&path2)); assert_eq!(cache.len(), 2); @@ -724,9 +627,9 @@ mod tests { // Wait for TTL to expire thread::sleep(Duration::from_millis(150)); - // Entries should now return None and be removed when observed through get or contains_key - assert!(cache.get(&path1).is_none()); - assert_eq!(cache.len(), 1); // path1 was removed by get() + // Entries should now return None when observed through contains_key + assert!(!cache.contains_key(&path1)); + assert_eq!(cache.len(), 1); // path1 was removed by contains_key() assert!(!cache.contains_key(&path2)); assert_eq!(cache.len(), 0); // path2 was removed by contains_key() } @@ -757,7 +660,30 @@ mod tests { mock_time.inc(Duration::from_millis(151)); assert!(!cache.contains_key(&path2)); // Expired - assert!(cache.contains_key(&path3)); // Still valid + assert!(cache.contains_key(&path3)); // Still valid + } + + #[test] + fn test_ttl_expiration_in_get() { + let ttl = Duration::from_millis(100); + let cache = DefaultListFilesCache::new(10000, Some(ttl)); + + let (path, value, _) = create_test_list_files_entry("path", 2, 50); + + // Cache the entry + cache.put(&path, value.clone()); + + // Entry should be accessible immediately + let result = cache.get(&path); + assert!(result.is_some()); + assert_eq!(result.unwrap().files.len(), 2); + + // Wait for TTL to expire + thread::sleep(Duration::from_millis(150)); + + // Get should return None because entry expired + let result2 = cache.get(&path); + assert!(result2.is_none()); } #[test] @@ -806,28 +732,29 @@ mod tests { #[test] fn test_entry_creation() { // Test with empty vector - let empty_vec: Arc> = Arc::new(vec![]); + let empty_list = CachedFileList::new(vec![]); let now = Instant::now(); - let entry = ListFilesEntry::try_new(empty_vec, None, now); + let entry = ListFilesEntry::try_new(empty_list, None, now); assert!(entry.is_none()); // Validate entry size let metas: Vec = (0..5) .map(|i| create_test_object_meta(&format!("file{i}"), 30)) .collect(); - let metas = Arc::new(metas); - let entry = ListFilesEntry::try_new(metas, None, now).unwrap(); - assert_eq!(entry.metas.len(), 5); + let cached_list = CachedFileList::new(metas); + let entry = ListFilesEntry::try_new(cached_list, None, now).unwrap(); + assert_eq!(entry.cached_file_list.files.len(), 5); // Size should be: capacity * sizeof(ObjectMeta) + (5 * 30) for heap bytes - let expected_size = - (entry.metas.capacity() * size_of::()) + (entry.metas.len() * 30); + let expected_size = (entry.cached_file_list.files.capacity() + * size_of::()) + + (entry.cached_file_list.files.len() * 30); assert_eq!(entry.size_bytes, expected_size); // Test with TTL let meta = create_test_object_meta("file", 50); let ttl = Duration::from_secs(10); - let entry = - ListFilesEntry::try_new(Arc::new(vec![meta]), Some(ttl), now).unwrap(); + let cached_list = CachedFileList::new(vec![meta]); + let entry = ListFilesEntry::try_new(cached_list, Some(ttl), now).unwrap(); assert!(entry.expires.unwrap() > now); } @@ -872,7 +799,7 @@ mod tests { } } - // Prefix-aware cache tests + // Prefix filtering tests using CachedFileList::filter_by_prefix /// Helper function to create ObjectMeta with a specific location path fn create_object_meta_with_path(location: &str) -> ObjectMeta { @@ -888,30 +815,27 @@ mod tests { } #[test] - fn test_prefix_aware_cache_hit() { - // Scenario: Cache has full table listing, query for partition returns filtered results + fn test_prefix_filtering() { let cache = DefaultListFilesCache::new(100000, None); // Create files for a partitioned table let table_base = Path::from("my_table"); - let files = Arc::new(vec![ + let files = vec![ create_object_meta_with_path("my_table/a=1/file1.parquet"), create_object_meta_with_path("my_table/a=1/file2.parquet"), create_object_meta_with_path("my_table/a=2/file3.parquet"), create_object_meta_with_path("my_table/a=2/file4.parquet"), - ]); + ]; // Cache the full table listing - cache.put(&table_base, files); + let cached = CachedFileList::new(files); + cache.put(&table_base, cached); - // Query for partition a=1 using get_with_extra - // New API: get_with_extra(table_base, Some(relative_prefix)) - let prefix_a1 = Some(Path::from("a=1")); - let result = cache.get_with_extra(&table_base, &prefix_a1); + let result = cache.get(&table_base).unwrap(); - // Should return filtered results (only files from a=1) - assert!(result.is_some()); - let filtered = result.unwrap(); + // Filter for partition a=1 + let prefix_a1 = Some(Path::from("my_table/a=1")); + let filtered = result.filter_by_prefix(&prefix_a1); assert_eq!(filtered.len(), 2); assert!( filtered @@ -919,92 +843,46 @@ mod tests { .all(|m| m.location.as_ref().starts_with("my_table/a=1")) ); - // Query for partition a=2 - let prefix_a2 = Some(Path::from("a=2")); - let result_2 = cache.get_with_extra(&table_base, &prefix_a2); - - assert!(result_2.is_some()); - let filtered_2 = result_2.unwrap(); + // Filter for partition a=2 + let prefix_a2 = Some(Path::from("my_table/a=2")); + let filtered_2 = result.filter_by_prefix(&prefix_a2); assert_eq!(filtered_2.len(), 2); assert!( filtered_2 .iter() .all(|m| m.location.as_ref().starts_with("my_table/a=2")) ); - } - - #[test] - fn test_prefix_aware_cache_no_filter_returns_all() { - // Scenario: Query with no prefix filter should return all files - let cache = DefaultListFilesCache::new(100000, None); - - let table_base = Path::from("my_table"); - - // Cache full table listing with 4 files - let full_files = Arc::new(vec![ - create_object_meta_with_path("my_table/a=1/file1.parquet"), - create_object_meta_with_path("my_table/a=1/file2.parquet"), - create_object_meta_with_path("my_table/a=2/file3.parquet"), - create_object_meta_with_path("my_table/a=2/file4.parquet"), - ]); - cache.put(&table_base, full_files); - - // Query with no prefix filter (None) should return all 4 files - let result = cache.get_with_extra(&table_base, &None); - assert!(result.is_some()); - let files = result.unwrap(); - assert_eq!(files.len(), 4); - - // Also test using get() which delegates to get_with_extra(&None) - let result_get = cache.get(&table_base); - assert!(result_get.is_some()); - assert_eq!(result_get.unwrap().len(), 4); - } - - #[test] - fn test_prefix_aware_cache_miss_no_entry() { - // Scenario: Table not cached, query should miss - let cache = DefaultListFilesCache::new(100000, None); - - let table_base = Path::from("my_table"); - // Query for full table should miss (nothing cached) - let result = cache.get_with_extra(&table_base, &None); - assert!(result.is_none()); - - // Query with prefix should also miss - let prefix = Some(Path::from("a=1")); - let result_2 = cache.get_with_extra(&table_base, &prefix); - assert!(result_2.is_none()); + // No filter returns all + let all = result.filter_by_prefix(&None); + assert_eq!(all.len(), 4); } #[test] - fn test_prefix_aware_cache_no_matching_files() { - // Scenario: Cache has table listing but no files match the requested partition + fn test_prefix_no_matching_files() { let cache = DefaultListFilesCache::new(100000, None); let table_base = Path::from("my_table"); - let files = Arc::new(vec![ + let files = vec![ create_object_meta_with_path("my_table/a=1/file1.parquet"), create_object_meta_with_path("my_table/a=2/file2.parquet"), - ]); - cache.put(&table_base, files); + ]; - // Query for partition a=3 which doesn't exist - let prefix_a3 = Some(Path::from("a=3")); - let result = cache.get_with_extra(&table_base, &prefix_a3); + cache.put(&table_base, CachedFileList::new(files)); + let result = cache.get(&table_base).unwrap(); - // Should return None since no files match - assert!(result.is_none()); + // Query for partition a=3 which doesn't exist + let prefix_a3 = Some(Path::from("my_table/a=3")); + let filtered = result.filter_by_prefix(&prefix_a3); + assert!(filtered.is_empty()); } #[test] - fn test_prefix_aware_nested_partitions() { - // Scenario: Table with multiple partition levels (e.g., year/month/day) + fn test_nested_partitions() { let cache = DefaultListFilesCache::new(100000, None); let table_base = Path::from("events"); - let files = Arc::new(vec![ + let files = vec![ create_object_meta_with_path( "events/year=2024/month=01/day=01/file1.parquet", ), @@ -1017,56 +895,19 @@ mod tests { create_object_meta_with_path( "events/year=2025/month=01/day=01/file4.parquet", ), - ]); - cache.put(&table_base, files); + ]; - // Query for year=2024/month=01 (should get 2 files) - let prefix_month = Some(Path::from("year=2024/month=01")); - let result = cache.get_with_extra(&table_base, &prefix_month); - assert!(result.is_some()); - assert_eq!(result.unwrap().len(), 2); - - // Query for year=2024 (should get 3 files) - let prefix_year = Some(Path::from("year=2024")); - let result_year = cache.get_with_extra(&table_base, &prefix_year); - assert!(result_year.is_some()); - assert_eq!(result_year.unwrap().len(), 3); - - // Query for specific day (should get 1 file) - let prefix_day = Some(Path::from("year=2024/month=01/day=01")); - let result_day = cache.get_with_extra(&table_base, &prefix_day); - assert!(result_day.is_some()); - assert_eq!(result_day.unwrap().len(), 1); - } + cache.put(&table_base, CachedFileList::new(files)); + let result = cache.get(&table_base).unwrap(); - #[test] - fn test_prefix_aware_different_tables() { - // Scenario: Multiple tables cached, queries should not cross-contaminate - let cache = DefaultListFilesCache::new(100000, None); + // Filter for year=2024/month=01 + let prefix_month = Some(Path::from("events/year=2024/month=01")); + let filtered = result.filter_by_prefix(&prefix_month); + assert_eq!(filtered.len(), 2); - let table_a = Path::from("table_a"); - let table_b = Path::from("table_b"); - - let files_a = Arc::new(vec![create_object_meta_with_path( - "table_a/part=1/file1.parquet", - )]); - let files_b = Arc::new(vec![ - create_object_meta_with_path("table_b/part=1/file1.parquet"), - create_object_meta_with_path("table_b/part=2/file2.parquet"), - ]); - - cache.put(&table_a, files_a); - cache.put(&table_b, files_b); - - // Query table_a should only return table_a files - let result_a = cache.get(&table_a); - assert!(result_a.is_some()); - assert_eq!(result_a.unwrap().len(), 1); - - // Query table_b with prefix should only return matching table_b files - let prefix = Some(Path::from("part=1")); - let result_b = cache.get_with_extra(&table_b, &prefix); - assert!(result_b.is_some()); - assert_eq!(result_b.unwrap().len(), 1); + // Filter for year=2024 + let prefix_year = Some(Path::from("events/year=2024")); + let filtered_year = result.filter_by_prefix(&prefix_year); + assert_eq!(filtered_year.len(), 3); } } diff --git a/datafusion/execution/src/cache/mod.rs b/datafusion/execution/src/cache/mod.rs index 8172069fdbabd..da33fa10afed4 100644 --- a/datafusion/execution/src/cache/mod.rs +++ b/datafusion/execution/src/cache/mod.rs @@ -15,6 +15,8 @@ // specific language governing permissions and limitations // under the License. +use std::future::Future; + pub mod cache_manager; pub mod cache_unit; pub mod lru_queue; @@ -28,32 +30,89 @@ pub use list_files_cache::DefaultListFilesCache; /// A trait that can be implemented to provide custom cache behavior for the caches managed by /// [`cache_manager::CacheManager`]. /// +/// This trait provides a single atomic `get_or_update_with` method that handles both +/// cache retrieval and population in one operation. The closure receives the existing +/// cached value (if any) and returns the value to cache. +/// +/// ## Note on dyn-compatibility +/// +/// This trait is NOT dyn-compatible due to the generic type parameters on `get_or_update_with`. +/// For dyn-compatible cache traits, use the specific cache traits like [`cache_manager::FileStatisticsCache`], +/// [`cache_manager::ListFilesCache`], or [`cache_manager::FileMetadataCache`] which use `async_trait`. +/// +/// ## Validation +/// +/// Validation metadata (e.g., file size, last modified time) should be embedded in the +/// value type `V`. The closure is responsible for validating stale entries by comparing +/// the cached metadata against the current state. +/// +/// ## Thread Safety +/// /// Implementations must handle their own locking via internal mutability, as methods do not /// take mutable references and may be accessed by multiple concurrent queries. +/// +/// ## Async Safety with DashMap +/// +/// When implementing with DashMap, ensure locks are NOT held across await points to avoid +/// [potential deadlocks](https://dev.to/acter/beware-of-the-dashmap-deadlock-lij). +/// The correct pattern is: +/// 1. Get existing value (lock acquired and released immediately) +/// 2. Await the closure (no lock held) +/// 3. Insert result (lock acquired and released immediately) pub trait CacheAccessor: Send + Sync { - // Extra info but not part of the cache key or cache value. - type Extra: Clone; - - /// Get value from cache. - fn get(&self, k: &K) -> Option; - /// Get value from cache. - fn get_with_extra(&self, k: &K, e: &Self::Extra) -> Option; - /// Put value into cache. Returns the old value associated with the key if there was one. - fn put(&self, key: &K, value: V) -> Option; - /// Put value into cache. Returns the old value associated with the key if there was one. - fn put_with_extra(&self, key: &K, value: V, e: &Self::Extra) -> Option; - /// Remove an entry from the cache, returning value if they existed in the map. + /// Atomically get or update a cache entry. + /// + /// The closure receives `Option`: + /// - `Some(cached_value)` if something is cached (may be stale - closure should validate) + /// - `None` if not cached + /// + /// The closure should: + /// - Validate the existing value against current state (if applicable) + /// - Return the value to cache (can return existing unchanged, or compute new) + /// + /// Returns the value from the closure, which is also stored in the cache. + /// + /// # Example + /// + /// ```ignore + /// let result = cache.get_or_update_with(&key, |existing| async move { + /// // Check if cached and still valid + /// if let Some(cached) = existing.filter(|c| c.is_valid_for(¤t_meta)) { + /// return Ok(cached); + /// } + /// // Cache miss or invalid - compute new value + /// let new_value = compute_value().await?; + /// Ok(new_value) + /// }).await?; + /// ``` + fn get_or_update_with( + &self, + key: &K, + f: F, + ) -> impl Future> + Send + where + F: FnOnce(Option) -> Fut + Send, + Fut: Future> + Send, + E: Send + 'static, + V: Clone + Send; + + /// Remove an entry from the cache, returning the value if it existed. fn remove(&self, k: &K) -> Option; + /// Check if the cache contains a specific key. fn contains_key(&self, k: &K) -> bool; + /// Fetch the total number of cache entries. fn len(&self) -> usize; - /// Check if the Cache collection is empty or not. + + /// Check if the cache collection is empty. fn is_empty(&self) -> bool { self.len() == 0 } + /// Remove all entries from the cache. fn clear(&self); + /// Return the cache name. fn name(&self) -> String; } diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index fa961981c0488..0fbccb5a3c319 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -458,6 +458,21 @@ impl LexOrdering { }) } + /// Checks if `other` is a prefix of this `LexOrdering`. + pub fn is_prefix(&self, other: &LexOrdering) -> bool { + let self_exprs = self.as_ref(); + let other_exprs = other.as_ref(); + + if other_exprs.len() > self_exprs.len() { + return false; + } + + other_exprs + .iter() + .zip(self_exprs.iter()) + .all(|(req, cur)| req.expr.eq(&cur.expr) && req.options == cur.options) + } + /// Returns the sort options for the given expression if one is defined in this `LexOrdering`. pub fn get_sort_options(&self, expr: &dyn PhysicalExpr) -> Option { for e in self { @@ -803,6 +818,130 @@ impl DerefMut for OrderingRequirements { #[cfg(test)] mod tests { use super::*; + use arrow::datatypes::{DataType, FieldRef, Schema}; + use arrow::record_batch::RecordBatch; + use datafusion_common::Result; + use datafusion_expr_common::columnar_value::ColumnarValue; + use std::any::Any; + use std::fmt::{Display, Formatter}; + + /// A simple mock PhysicalExpr for testing, identified by name. + #[derive(Debug, Clone, PartialEq, Eq, Hash)] + struct TestExpr { + name: &'static str, + } + + impl TestExpr { + fn new_expr(name: &'static str) -> Arc { + Arc::new(Self { name }) + } + } + + impl Display for TestExpr { + fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { + write!(f, "{}", self.name) + } + } + + impl PhysicalExpr for TestExpr { + fn as_any(&self) -> &dyn Any { + self + } + + fn data_type(&self, _input_schema: &Schema) -> Result { + Ok(DataType::Int32) + } + + fn nullable(&self, _input_schema: &Schema) -> Result { + Ok(true) + } + + fn evaluate(&self, _batch: &RecordBatch) -> Result { + unimplemented!("TestExpr::evaluate is not needed for sort tests") + } + + fn children(&self) -> Vec<&Arc> { + vec![] + } + + fn with_new_children( + self: Arc, + _children: Vec>, + ) -> Result> { + Ok(self) + } + + fn fmt_sql(&self, f: &mut Formatter<'_>) -> fmt::Result { + write!(f, "{}", self.name) + } + + fn return_field(&self, _input_schema: &Schema) -> Result { + unimplemented!("TestExpr::return_field is not needed for sort tests") + } + } + + /// Helper to create a PhysicalSortExpr with the given expression and options. + fn sort_expr(expr: Arc, options: SortOptions) -> PhysicalSortExpr { + PhysicalSortExpr::new(expr, options) + } + + #[test] + fn test_is_prefix() { + let asc = SortOptions { + descending: false, + nulls_first: false, + }; + let desc = SortOptions { + descending: true, + nulls_first: true, + }; + + let a = TestExpr::new_expr("a"); + let b = TestExpr::new_expr("b"); + let c = TestExpr::new_expr("c"); + + // [a ASC] is a prefix of [a ASC] (exact match) + let ordering1 = LexOrdering::new([sort_expr(Arc::clone(&a), asc)]).unwrap(); + let ordering2 = LexOrdering::new([sort_expr(Arc::clone(&a), asc)]).unwrap(); + assert!(ordering1.is_prefix(&ordering2)); + + // [a ASC] is a prefix of [a ASC, b DESC] + let ordering_ab = LexOrdering::new([ + sort_expr(Arc::clone(&a), asc), + sort_expr(Arc::clone(&b), desc), + ]) + .unwrap(); + assert!(ordering_ab.is_prefix(&ordering1)); + + // [a ASC, b DESC] is NOT a prefix of [a ASC] (other is longer) + assert!(!ordering1.is_prefix(&ordering_ab)); + + // [a DESC] is NOT a prefix of [a ASC] (different sort options) + let ordering_a_desc = + LexOrdering::new([sort_expr(Arc::clone(&a), desc)]).unwrap(); + assert!(!ordering1.is_prefix(&ordering_a_desc)); + + // [b ASC] is NOT a prefix of [a ASC] (different expressions) + let ordering_b = LexOrdering::new([sort_expr(Arc::clone(&b), asc)]).unwrap(); + assert!(!ordering1.is_prefix(&ordering_b)); + + // [a ASC, b ASC] is a prefix of [a ASC, b ASC, c ASC] + let ordering_ab_asc = LexOrdering::new([ + sort_expr(Arc::clone(&a), asc), + sort_expr(Arc::clone(&b), asc), + ]) + .unwrap(); + let ordering_abc = LexOrdering::new([ + sort_expr(Arc::clone(&a), asc), + sort_expr(Arc::clone(&b), asc), + sort_expr(Arc::clone(&c), asc), + ]) + .unwrap(); + assert!(ordering_abc.is_prefix(&ordering_ab_asc)); + + // [a ASC, b DESC] is NOT a prefix of [a ASC, b ASC, c ASC] (mismatch in middle) + assert!(!ordering_abc.is_prefix(&ordering_ab)); + } #[test] fn test_is_reversed_sort_options() { diff --git a/datafusion/proto/src/physical_plan/from_proto.rs b/datafusion/proto/src/physical_plan/from_proto.rs index 073fdd858cdd3..9f4c84f2775e9 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -616,30 +616,33 @@ impl TryFrom<&protobuf::PartitionedFile> for PartitionedFile { type Error = DataFusionError; fn try_from(val: &protobuf::PartitionedFile) -> Result { - Ok(PartitionedFile { - object_meta: ObjectMeta { - location: Path::parse(val.path.as_str()).map_err(|e| { - proto_error(format!("Invalid object_store path: {e}")) - })?, - last_modified: Utc.timestamp_nanos(val.last_modified_ns as i64), - size: val.size, - e_tag: None, - version: None, - }, - partition_values: val - .partition_values + let range = val.range.as_ref().map(|v| (v.start, v.end)); + let statistics = val + .statistics + .as_ref() + .map(|v| v.try_into().map(Arc::new)) + .transpose()?; + let mut file = PartitionedFile::new_from_meta(ObjectMeta { + location: Path::parse(val.path.as_str()) + .map_err(|e| proto_error(format!("Invalid object_store path: {e}")))?, + last_modified: Utc.timestamp_nanos(val.last_modified_ns as i64), + size: val.size, + e_tag: None, + version: None, + }) + .with_partition_values( + val.partition_values .iter() .map(|v| v.try_into()) .collect::, _>>()?, - range: val.range.as_ref().map(|v| v.try_into()).transpose()?, - statistics: val - .statistics - .as_ref() - .map(|v| v.try_into().map(Arc::new)) - .transpose()?, - extensions: None, - metadata_size_hint: None, - }) + ); + if let Some((start, end)) = range { + file = file.with_range(start, end); + } + if let Some(statistics) = statistics { + file = file.with_statistics(statistics); + } + Ok(file) } } @@ -754,20 +757,13 @@ mod tests { #[test] fn partitioned_file_path_roundtrip_percent_encoded() { let path_str = "foo/foo%2Fbar/baz%252Fqux"; - let pf = PartitionedFile { - object_meta: ObjectMeta { - location: Path::parse(path_str).unwrap(), - last_modified: Utc.timestamp_nanos(1_000), - size: 42, - e_tag: None, - version: None, - }, - partition_values: vec![], - range: None, - statistics: None, - extensions: None, - metadata_size_hint: None, - }; + let pf = PartitionedFile::new_from_meta(ObjectMeta { + location: Path::parse(path_str).unwrap(), + last_modified: Utc.timestamp_nanos(1_000), + size: 42, + e_tag: None, + version: None, + }); let proto = protobuf::PartitionedFile::try_from(&pf).unwrap(); assert_eq!(proto.path, path_str); diff --git a/datafusion/sqllogictest/test_files/parquet.slt b/datafusion/sqllogictest/test_files/parquet.slt index be713b963b451..ab57f1301f194 100644 --- a/datafusion/sqllogictest/test_files/parquet.slt +++ b/datafusion/sqllogictest/test_files/parquet.slt @@ -889,3 +889,262 @@ WHERE b = 2; statement ok DROP TABLE t; + +################### +# Test: Ordering inference from Parquet sorting_columns metadata +# Verifies DataFusion auto-infers ordering WITHOUT needing WITH ORDER clause +################### + +# Create source data +statement ok +CREATE TABLE ordering_src AS VALUES (1, 10), (2, 20), (3, 30), (4, 40), (5, 50); + +# Write sorted Parquet file (COPY TO with ORDER BY writes sorting_columns metadata) +query I +COPY (SELECT column1 as a, column2 as b FROM ordering_src ORDER BY a ASC NULLS FIRST) +TO 'test_files/scratch/parquet/ordering_inference.parquet' +STORED AS PARQUET; +---- +5 + +# Create external table pointing to sorted Parquet (NO WITH ORDER clause!) +statement ok +CREATE EXTERNAL TABLE sorted_inferred (a INT, b INT) +STORED AS PARQUET +LOCATION 'test_files/scratch/parquet/ordering_inference.parquet'; + +# Verify ordering is inferred - expect output_ordering in DataSourceExec, NO SortExec +query TT +EXPLAIN SELECT a, b FROM sorted_inferred ORDER BY a ASC NULLS FIRST; +---- +logical_plan +01)Sort: sorted_inferred.a ASC NULLS FIRST +02)--TableScan: sorted_inferred projection=[a, b] +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/ordering_inference.parquet]]}, projection=[a, b], output_ordering=[a@0 ASC], file_type=parquet + +# Cleanup +statement ok +DROP TABLE sorted_inferred; + +statement ok +DROP TABLE ordering_src; + +################### +# Test: DESC ordering inference from Parquet sorting_columns metadata +################### + +statement ok +CREATE TABLE ordering_desc_src AS VALUES (5, 50), (4, 40), (3, 30), (2, 20), (1, 10); + +query I +COPY (SELECT column1 as a, column2 as b FROM ordering_desc_src ORDER BY a DESC NULLS LAST) +TO 'test_files/scratch/parquet/ordering_desc.parquet' +STORED AS PARQUET; +---- +5 + +statement ok +CREATE EXTERNAL TABLE sorted_desc (a INT, b INT) +STORED AS PARQUET +LOCATION 'test_files/scratch/parquet/ordering_desc.parquet'; + +# Verify DESC ordering is inferred - no SortExec needed +query TT +EXPLAIN SELECT a, b FROM sorted_desc ORDER BY a DESC NULLS LAST; +---- +logical_plan +01)Sort: sorted_desc.a DESC NULLS LAST +02)--TableScan: sorted_desc projection=[a, b] +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/ordering_desc.parquet]]}, projection=[a, b], output_ordering=[a@0 DESC NULLS LAST], file_type=parquet + +statement ok +DROP TABLE sorted_desc; + +statement ok +DROP TABLE ordering_desc_src; + +################### +# Test: Reverse ordering - file is DESC but query wants ASC +# Ordering is inferred with reverse_order, but SortExec is still needed +################### + +statement ok +CREATE TABLE ordering_reverse_src AS VALUES (5, 50), (4, 40), (3, 30), (2, 20), (1, 10); + +query I +COPY (SELECT column1 as a, column2 as b FROM ordering_reverse_src ORDER BY a DESC NULLS LAST) +TO 'test_files/scratch/parquet/ordering_reverse.parquet' +STORED AS PARQUET; +---- +5 + +statement ok +CREATE EXTERNAL TABLE sorted_reverse (a INT, b INT) +STORED AS PARQUET +LOCATION 'test_files/scratch/parquet/ordering_reverse.parquet'; + +# Query wants ASC but file is DESC - ordering is inferred with reverse_order, SortExec preserved +query TT +EXPLAIN SELECT a, b FROM sorted_reverse ORDER BY a ASC NULLS FIRST; +---- +logical_plan +01)Sort: sorted_reverse.a ASC NULLS FIRST +02)--TableScan: sorted_reverse projection=[a, b] +physical_plan +01)SortExec: expr=[a@0 ASC], preserve_partitioning=[false] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/ordering_reverse.parquet]]}, projection=[a, b], file_type=parquet, reverse_row_groups=true + +statement ok +DROP TABLE sorted_reverse; + +statement ok +DROP TABLE ordering_reverse_src; + +################### +# Test: Multi-column ordering inference from Parquet sorting_columns metadata +################### + +statement ok +CREATE TABLE ordering_multi_src AS VALUES (1, 10, 100), (1, 20, 200), (2, 10, 300), (2, 20, 400); + +query I +COPY (SELECT column1 as a, column2 as b, column3 as c FROM ordering_multi_src ORDER BY a ASC NULLS FIRST, b ASC NULLS FIRST) +TO 'test_files/scratch/parquet/ordering_multi.parquet' +STORED AS PARQUET; +---- +4 + +statement ok +CREATE EXTERNAL TABLE sorted_multi (a INT, b INT, c INT) +STORED AS PARQUET +LOCATION 'test_files/scratch/parquet/ordering_multi.parquet'; + +# Verify multi-column ordering is inferred - no SortExec needed +query TT +EXPLAIN SELECT a, b, c FROM sorted_multi ORDER BY a ASC NULLS FIRST, b ASC NULLS FIRST; +---- +logical_plan +01)Sort: sorted_multi.a ASC NULLS FIRST, sorted_multi.b ASC NULLS FIRST +02)--TableScan: sorted_multi projection=[a, b, c] +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/ordering_multi.parquet]]}, projection=[a, b, c], output_ordering=[a@0 ASC, b@1 ASC], file_type=parquet + +statement ok +DROP TABLE sorted_multi; + +statement ok +DROP TABLE ordering_multi_src; + +################### +# Test: Files with no ordering metadata should not infer ordering +################### + +statement ok +CREATE TABLE no_ordering_src AS VALUES (3, 30), (1, 10), (2, 20); + +# Write Parquet WITHOUT ORDER BY - no sorting_columns metadata +query I +COPY (SELECT column1 as a, column2 as b FROM no_ordering_src) +TO 'test_files/scratch/parquet/no_ordering.parquet' +STORED AS PARQUET; +---- +3 + +statement ok +CREATE EXTERNAL TABLE no_ordering (a INT, b INT) +STORED AS PARQUET +LOCATION 'test_files/scratch/parquet/no_ordering.parquet'; + +# Verify NO ordering is inferred - SortExec IS required +query TT +EXPLAIN SELECT a, b FROM no_ordering ORDER BY a ASC NULLS FIRST; +---- +logical_plan +01)Sort: no_ordering.a ASC NULLS FIRST +02)--TableScan: no_ordering projection=[a, b] +physical_plan +01)SortExec: expr=[a@0 ASC], preserve_partitioning=[false] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/no_ordering.parquet]]}, projection=[a, b], file_type=parquet + +statement ok +DROP TABLE no_ordering; + +statement ok +DROP TABLE no_ordering_src; + +################### +# Test: Ordering inference with nested/complex types +# Parquet can store sorting_columns metadata for nested types. +# This tests whether DataFusion handles such cases correctly. +################### + +# Test 1: Struct type - Parquet allows sorting on struct columns +query I +COPY (SELECT {key: i, value: i * 10} as nested_col FROM generate_series(1, 5) as t(i) ORDER BY nested_col) +TO 'test_files/scratch/parquet/ordering_struct.parquet' +STORED AS PARQUET; +---- +5 + +statement ok +CREATE EXTERNAL TABLE sorted_struct +STORED AS PARQUET +LOCATION 'test_files/scratch/parquet/ordering_struct.parquet'; + +# Query the data to verify it was written +query ? +SELECT nested_col FROM sorted_struct; +---- +{key: 1, value: 10} +{key: 2, value: 20} +{key: 3, value: 30} +{key: 4, value: 40} +{key: 5, value: 50} + +statement ok +DROP TABLE sorted_struct; + +# Test 2: Array type - Arrays can be sorted lexicographically +query I +COPY (SELECT make_array(i, i+1, i+2) as arr_col FROM generate_series(1, 3) as t(i) ORDER BY arr_col) +TO 'test_files/scratch/parquet/ordering_array.parquet' +STORED AS PARQUET; +---- +3 + +statement ok +CREATE EXTERNAL TABLE sorted_array +STORED AS PARQUET +LOCATION 'test_files/scratch/parquet/ordering_array.parquet'; + +query ? +SELECT arr_col FROM sorted_array; +---- +[1, 2, 3] +[2, 3, 4] +[3, 4, 5] + +statement ok +DROP TABLE sorted_array; + +# Test 3: Nested struct with array +query I +COPY (SELECT {key: i, values: make_array(i, i*2)} as complex_col FROM generate_series(1, 3) as t(i) ORDER BY complex_col) +TO 'test_files/scratch/parquet/ordering_complex.parquet' +STORED AS PARQUET; +---- +3 + +statement ok +CREATE EXTERNAL TABLE sorted_complex +STORED AS PARQUET +LOCATION 'test_files/scratch/parquet/ordering_complex.parquet'; + +query ? +SELECT complex_col FROM sorted_complex; +---- +{key: 1, values: [1, 2]} +{key: 2, values: [2, 4]} +{key: 3, values: [3, 6]} + +statement ok +DROP TABLE sorted_complex; diff --git a/datafusion/substrait/src/physical_plan/consumer.rs b/datafusion/substrait/src/physical_plan/consumer.rs index ac0f26722513c..ccaf1abec4245 100644 --- a/datafusion/substrait/src/physical_plan/consumer.rs +++ b/datafusion/substrait/src/physical_plan/consumer.rs @@ -119,20 +119,14 @@ pub async fn from_substrait_rel( .unwrap(); let size = 0; - let partitioned_file = PartitionedFile { - object_meta: ObjectMeta { + let partitioned_file = + PartitionedFile::new_from_meta(ObjectMeta { last_modified: last_modified.into(), location: path.into(), size, e_tag: None, version: None, - }, - partition_values: vec![], - range: None, - statistics: None, - extensions: None, - metadata_size_hint: None, - }; + }); let part_index = file.partition_index as usize; while part_index >= file_groups.len() {