From b1eac733ec81e6dc334de65da1e3a1501df8d627 Mon Sep 17 00:00:00 2001 From: logan-keede Date: Wed, 5 Feb 2025 23:33:12 +0530 Subject: [PATCH 1/7] First Iteration --- .../examples/custom_file_format.rs | 15 +- datafusion/catalog-listing/Cargo.toml | 7 +- .../catalog-listing/src/file_scan_config.rs | 1616 +++++++++++++++++ .../catalog-listing/src/minmax_statistics.rs | 287 +++ .../core/src/datasource/file_format/arrow.rs | 12 +- .../core/src/datasource/file_format/avro.rs | 11 +- .../core/src/datasource/file_format/csv.rs | 14 +- .../core/src/datasource/file_format/json.rs | 12 +- .../core/src/datasource/file_format/mod.rs | 18 +- .../src/datasource/file_format/parquet.rs | 16 +- .../core/src/datasource/listing/table.rs | 15 +- .../physical_plan/file_scan_config.rs | 6 +- .../datasource/physical_plan/file_stream.rs | 2 +- .../core/src/datasource/physical_plan/mod.rs | 307 +--- .../datasource/physical_plan/statistics.rs | 2 +- .../core/tests/catalog_listing/helpers.rs | 224 +++ datafusion/core/tests/catalog_listing/mod.rs | 18 + datafusion/core/tests/core_integration.rs | 2 + 18 files changed, 2234 insertions(+), 350 deletions(-) create mode 100644 datafusion/catalog-listing/src/file_scan_config.rs create mode 100644 datafusion/catalog-listing/src/minmax_statistics.rs create mode 100644 datafusion/core/tests/catalog_listing/helpers.rs create mode 100644 datafusion/core/tests/catalog_listing/mod.rs diff --git a/datafusion-examples/examples/custom_file_format.rs b/datafusion-examples/examples/custom_file_format.rs index 5493fa21968e..aa2df036235a 100644 --- a/datafusion-examples/examples/custom_file_format.rs +++ b/datafusion-examples/examples/custom_file_format.rs @@ -21,10 +21,13 @@ use arrow::{ array::{AsArray, RecordBatch, StringArray, UInt8Array}, datatypes::{DataType, Field, Schema, SchemaRef, UInt64Type}, }; -use datafusion::common::{GetExt, Statistics}; use datafusion::execution::session_state::SessionStateBuilder; use datafusion::physical_expr::LexRequirement; use datafusion::physical_expr::PhysicalExpr; +use datafusion::{ + catalog::Session, + common::{GetExt, Statistics}, +}; use datafusion::{ datasource::{ file_format::{ @@ -83,7 +86,7 @@ impl FileFormat for TSVFileFormat { async fn infer_schema( &self, - state: &SessionState, + state: &dyn Session, store: &Arc, objects: &[ObjectMeta], ) -> Result { @@ -94,7 +97,7 @@ impl FileFormat for TSVFileFormat { async fn infer_stats( &self, - state: &SessionState, + state: &dyn Session, store: &Arc, table_schema: SchemaRef, object: &ObjectMeta, @@ -106,7 +109,7 @@ impl FileFormat for TSVFileFormat { async fn create_physical_plan( &self, - state: &SessionState, + state: &dyn Session, conf: FileScanConfig, filters: Option<&Arc>, ) -> Result> { @@ -118,7 +121,7 @@ impl FileFormat for TSVFileFormat { async fn create_writer_physical_plan( &self, input: Arc, - state: &SessionState, + state: &dyn Session, conf: FileSinkConfig, order_requirements: Option, ) -> Result> { @@ -148,7 +151,7 @@ impl TSVFileFactory { impl FileFormatFactory for TSVFileFactory { fn create( &self, - state: &SessionState, + state: &dyn Session, format_options: &std::collections::HashMap, ) -> Result> { let mut new_options = format_options.clone(); diff --git a/datafusion/catalog-listing/Cargo.toml b/datafusion/catalog-listing/Cargo.toml index 03132e7b7bb5..a0bb3de255e0 100644 --- a/datafusion/catalog-listing/Cargo.toml +++ b/datafusion/catalog-listing/Cargo.toml @@ -28,6 +28,7 @@ rust-version.workspace = true version.workspace = true [dependencies] +apache-avro = { version = "0.17", optional = true } arrow = { workspace = true } arrow-schema = { workspace = true } async-compression = { version = "0.4.0", features = [ @@ -49,11 +50,15 @@ futures = { workspace = true } glob = "0.3.0" itertools = { workspace = true } log = { workspace = true } +num-traits = { version = "0.2", optional = true } object_store = { workspace = true } url = { workspace = true } +[features] +# Used to enable the avro format +avro = ["apache-avro", "num-traits", "datafusion-common/avro"] + [dev-dependencies] -async-trait = { workspace = true } tempfile = { workspace = true } tokio = { workspace = true } diff --git a/datafusion/catalog-listing/src/file_scan_config.rs b/datafusion/catalog-listing/src/file_scan_config.rs new file mode 100644 index 000000000000..00c0ce98a6fb --- /dev/null +++ b/datafusion/catalog-listing/src/file_scan_config.rs @@ -0,0 +1,1616 @@ +// 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. + +//! [`FileScanConfig`] to configure scanning of possibly partitioned +//! file sources. + +use std::{ + borrow::Cow, + collections::HashMap, + fmt::{Debug, Formatter, Result as FmtResult}, + marker::PhantomData, + mem::size_of, + sync::Arc, + vec, +}; + +use crate::minmax_statistics::MinMaxStatistics; +use crate::PartitionedFile; +use datafusion_common::error::Result; +use datafusion_common::scalar::ScalarValue; +use datafusion_execution::object_store::ObjectStoreUrl; + +use arrow::array::{ArrayData, BufferBuilder}; +use arrow::array::{ArrayRef, DictionaryArray, RecordBatch, RecordBatchOptions}; +use arrow::buffer::Buffer; +use arrow::datatypes::{ArrowNativeType, UInt16Type}; +use arrow_schema::{DataType, Field, Schema, SchemaRef}; +use datafusion_common::stats::Precision; +use datafusion_common::{ + exec_err, ColumnStatistics, Constraints, DataFusionError, Statistics, +}; +use datafusion_physical_expr::expressions::Column; +use datafusion_physical_expr::{LexOrdering, PhysicalSortExpr}; + +use datafusion_physical_plan::display::display_orderings; +use datafusion_physical_plan::display::ProjectSchemaDisplay; +use datafusion_physical_plan::DisplayAs; +use datafusion_physical_plan::DisplayFormatType; +use log::{debug, warn}; + +/// A wrapper to customize partitioned file display +/// +/// Prints in the format: +/// ```text +/// {NUM_GROUPS groups: [[file1, file2,...], [fileN, fileM, ...], ...]} +/// ``` +#[derive(Debug)] +struct FileGroupsDisplay<'a>(&'a [Vec]); + +impl DisplayAs for FileGroupsDisplay<'_> { + fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> FmtResult { + let n_groups = self.0.len(); + let groups = if n_groups == 1 { "group" } else { "groups" }; + write!(f, "{{{n_groups} {groups}: [")?; + match t { + DisplayFormatType::Default => { + // To avoid showing too many partitions + let max_groups = 5; + fmt_up_to_n_elements(self.0, max_groups, f, |group, f| { + FileGroupDisplay(group).fmt_as(t, f) + })?; + } + DisplayFormatType::Verbose => { + fmt_elements_split_by_commas(self.0.iter(), f, |group, f| { + FileGroupDisplay(group).fmt_as(t, f) + })? + } + } + write!(f, "]}}") + } +} + +/// A wrapper to customize partitioned group of files display +/// +/// Prints in the format: +/// ```text +/// [file1, file2,...] +/// ``` +#[derive(Debug)] +pub struct FileGroupDisplay<'a>(pub &'a [PartitionedFile]); + +impl DisplayAs for FileGroupDisplay<'_> { + fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> FmtResult { + write!(f, "[")?; + match t { + DisplayFormatType::Default => { + // To avoid showing too many files + let max_files = 5; + fmt_up_to_n_elements(self.0, max_files, f, |pf, f| { + write!(f, "{}", pf.object_meta.location.as_ref())?; + if let Some(range) = pf.range.as_ref() { + write!(f, ":{}..{}", range.start, range.end)?; + } + Ok(()) + })? + } + DisplayFormatType::Verbose => { + fmt_elements_split_by_commas(self.0.iter(), f, |pf, f| { + write!(f, "{}", pf.object_meta.location.as_ref())?; + if let Some(range) = pf.range.as_ref() { + write!(f, ":{}..{}", range.start, range.end)?; + } + Ok(()) + })? + } + } + write!(f, "]") + } +} + +/// helper to format an array of up to N elements +fn fmt_up_to_n_elements( + elements: &[E], + n: usize, + f: &mut Formatter, + format_element: F, +) -> FmtResult +where + F: Fn(&E, &mut Formatter) -> FmtResult, +{ + let len = elements.len(); + fmt_elements_split_by_commas(elements.iter().take(n), f, |element, f| { + format_element(element, f) + })?; + // Remaining elements are showed as `...` (to indicate there is more) + if len > n { + write!(f, ", ...")?; + } + Ok(()) +} + +/// helper formatting array elements with a comma and a space between them +fn fmt_elements_split_by_commas( + iter: I, + f: &mut Formatter, + format_element: F, +) -> FmtResult +where + I: Iterator, + F: Fn(E, &mut Formatter) -> FmtResult, +{ + for (idx, element) in iter.enumerate() { + if idx > 0 { + write!(f, ", ")?; + } + format_element(element, f)?; + } + Ok(()) +} + +/// The various listing tables does not attempt to read all files +/// concurrently, instead they will read files in sequence within a +/// partition. This is an important property as it allows plans to +/// run against 1000s of files and not try to open them all +/// concurrently. +/// +/// However, it means if we assign more than one file to a partition +/// the output sort order will not be preserved as illustrated in the +/// following diagrams: +/// +/// When only 1 file is assigned to each partition, each partition is +/// correctly sorted on `(A, B, C)` +/// +/// ```text +///┏ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ┓ +/// ┌ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┐ ┌ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┌ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┌ ─ ─ ─ ─ ─ ─ ─ ─ ┐ +///┃ ┌───────────────┐ ┌──────────────┐ │ ┌──────────────┐ │ ┌─────────────┐ ┃ +/// │ │ 1.parquet │ │ │ │ 2.parquet │ │ │ 3.parquet │ │ │ 4.parquet │ │ +///┃ │ Sort: A, B, C │ │Sort: A, B, C │ │ │Sort: A, B, C │ │ │Sort: A, B, C│ ┃ +/// │ └───────────────┘ │ │ └──────────────┘ │ └──────────────┘ │ └─────────────┘ │ +///┃ │ │ ┃ +/// │ │ │ │ │ │ +///┃ │ │ ┃ +/// │ │ │ │ │ │ +///┃ │ │ ┃ +/// │ │ │ │ │ │ +///┃ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┘ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┘ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┃ +/// DataFusion DataFusion DataFusion DataFusion +///┃ Partition 1 Partition 2 Partition 3 Partition 4 ┃ +/// ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ +/// +/// ParquetExec +///``` +/// +/// However, when more than 1 file is assigned to each partition, each +/// partition is NOT correctly sorted on `(A, B, C)`. Once the second +/// file is scanned, the same values for A, B and C can be repeated in +/// the same sorted stream +/// +///```text +///┏ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ +/// ┌ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┐ ┌ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┃ +///┃ ┌───────────────┐ ┌──────────────┐ │ +/// │ │ 1.parquet │ │ │ │ 2.parquet │ ┃ +///┃ │ Sort: A, B, C │ │Sort: A, B, C │ │ +/// │ └───────────────┘ │ │ └──────────────┘ ┃ +///┃ ┌───────────────┐ ┌──────────────┐ │ +/// │ │ 3.parquet │ │ │ │ 4.parquet │ ┃ +///┃ │ Sort: A, B, C │ │Sort: A, B, C │ │ +/// │ └───────────────┘ │ │ └──────────────┘ ┃ +///┃ │ +/// │ │ │ ┃ +///┃ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┘ +/// DataFusion DataFusion ┃ +///┃ Partition 1 Partition 2 +/// ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ┛ +/// +/// ParquetExec +///``` +fn get_projected_output_ordering( + base_config: &FileScanConfig, + projected_schema: &SchemaRef, +) -> Vec { + let mut all_orderings = vec![]; + for output_ordering in &base_config.output_ordering { + let mut new_ordering = LexOrdering::default(); + for PhysicalSortExpr { expr, options } in output_ordering.iter() { + if let Some(col) = expr.as_any().downcast_ref::() { + let name = col.name(); + if let Some((idx, _)) = projected_schema.column_with_name(name) { + // Compute the new sort expression (with correct index) after projection: + new_ordering.push(PhysicalSortExpr { + expr: Arc::new(Column::new(name, idx)), + options: *options, + }); + continue; + } + } + // Cannot find expression in the projected_schema, stop iterating + // since rest of the orderings are violated + break; + } + + // do not push empty entries + // otherwise we may have `Some(vec![])` at the output ordering. + if new_ordering.is_empty() { + continue; + } + + // Check if any file groups are not sorted + if base_config.file_groups.iter().any(|group| { + if group.len() <= 1 { + // File groups with <= 1 files are always sorted + return false; + } + + let statistics = match MinMaxStatistics::new_from_files( + &new_ordering, + projected_schema, + base_config.projection.as_deref(), + group, + ) { + Ok(statistics) => statistics, + Err(e) => { + log::trace!("Error fetching statistics for file group: {e}"); + // we can't prove that it's ordered, so we have to reject it + return true; + } + }; + + !statistics.is_sorted() + }) { + debug!( + "Skipping specified output ordering {:?}. \ + Some file groups couldn't be determined to be sorted: {:?}", + base_config.output_ordering[0], base_config.file_groups + ); + continue; + } + + all_orderings.push(new_ordering); + } + all_orderings +} + +/// Convert type to a type suitable for use as a [`ListingTable`] +/// partition column. Returns `Dictionary(UInt16, val_type)`, which is +/// a reasonable trade off between a reasonable number of partition +/// values and space efficiency. +/// +/// This use this to specify types for partition columns. However +/// you MAY also choose not to dictionary-encode the data or to use a +/// different dictionary type. +/// +/// Use [`wrap_partition_value_in_dict`] to wrap a [`ScalarValue`] in the same say. +/// +/// [`ListingTable`]: crate::datasource::listing::ListingTable +pub fn wrap_partition_type_in_dict(val_type: DataType) -> DataType { + DataType::Dictionary(Box::new(DataType::UInt16), Box::new(val_type)) +} + +/// Convert a [`ScalarValue`] of partition columns to a type, as +/// described in the documentation of [`wrap_partition_type_in_dict`], +/// which can wrap the types. +pub fn wrap_partition_value_in_dict(val: ScalarValue) -> ScalarValue { + ScalarValue::Dictionary(Box::new(DataType::UInt16), Box::new(val)) +} + +/// The base configurations to provide when creating a physical plan for +/// any given file format. +/// +/// # Example +/// ``` +/// # use std::sync::Arc; +/// # use arrow_schema::Schema; +/// use datafusion::datasource::listing::PartitionedFile; +/// # use datafusion::datasource::physical_plan::FileScanConfig; +/// # use datafusion_execution::object_store::ObjectStoreUrl; +/// # let file_schema = Arc::new(Schema::empty()); +/// // create FileScan config for reading data from file:// +/// let object_store_url = ObjectStoreUrl::local_filesystem(); +/// let config = FileScanConfig::new(object_store_url, file_schema) +/// .with_limit(Some(1000)) // read only the first 1000 records +/// .with_projection(Some(vec![2, 3])) // project columns 2 and 3 +/// // Read /tmp/file1.parquet with known size of 1234 bytes in a single group +/// .with_file(PartitionedFile::new("file1.parquet", 1234)) +/// // Read /tmp/file2.parquet 56 bytes and /tmp/file3.parquet 78 bytes +/// // in a single row group +/// .with_file_group(vec![ +/// PartitionedFile::new("file2.parquet", 56), +/// PartitionedFile::new("file3.parquet", 78), +/// ]); +/// ``` +#[derive(Clone)] +pub struct FileScanConfig { + /// Object store URL, used to get an [`ObjectStore`] instance from + /// [`RuntimeEnv::object_store`] + /// + /// This `ObjectStoreUrl` should be the prefix of the absolute url for files + /// as `file://` or `s3://my_bucket`. It should not include the path to the + /// file itself. The relevant URL prefix must be registered via + /// [`RuntimeEnv::register_object_store`] + /// + /// [`ObjectStore`]: object_store::ObjectStore + /// [`RuntimeEnv::register_object_store`]: datafusion_execution::runtime_env::RuntimeEnv::register_object_store + /// [`RuntimeEnv::object_store`]: datafusion_execution::runtime_env::RuntimeEnv::object_store + pub object_store_url: ObjectStoreUrl, + /// Schema before `projection` is applied. It contains the all columns that may + /// appear in the files. It does not include table partition columns + /// that may be added. + pub file_schema: SchemaRef, + /// List of files to be processed, grouped into partitions + /// + /// Each file must have a schema of `file_schema` or a subset. If + /// a particular file has a subset, the missing columns are + /// padded with NULLs. + /// + /// DataFusion may attempt to read each partition of files + /// concurrently, however files *within* a partition will be read + /// sequentially, one after the next. + pub file_groups: Vec>, + /// Table constraints + pub constraints: Constraints, + /// Estimated overall statistics of the files, taking `filters` into account. + /// Defaults to [`Statistics::new_unknown`]. + pub statistics: Statistics, + /// Columns on which to project the data. Indexes that are higher than the + /// number of columns of `file_schema` refer to `table_partition_cols`. + pub projection: Option>, + /// The maximum number of records to read from this plan. If `None`, + /// all records after filtering are returned. + pub limit: Option, + /// The partitioning columns + pub table_partition_cols: Vec, + /// All equivalent lexicographical orderings that describe the schema. + pub output_ordering: Vec, +} + +impl FileScanConfig { + /// Create a new `FileScanConfig` with default settings for scanning files. + /// + /// See example on [`FileScanConfig`] + /// + /// No file groups are added by default. See [`Self::with_file`], [`Self::with_file_group]` and + /// [`Self::with_file_groups`]. + /// + /// # Parameters: + /// * `object_store_url`: See [`Self::object_store_url`] + /// * `file_schema`: See [`Self::file_schema`] + pub fn new(object_store_url: ObjectStoreUrl, file_schema: SchemaRef) -> Self { + let statistics = Statistics::new_unknown(&file_schema); + Self { + object_store_url, + file_schema, + file_groups: vec![], + constraints: Constraints::empty(), + statistics, + projection: None, + limit: None, + table_partition_cols: vec![], + output_ordering: vec![], + } + } + + /// Set the table constraints of the files + pub fn with_constraints(mut self, constraints: Constraints) -> Self { + self.constraints = constraints; + self + } + + /// Set the statistics of the files + pub fn with_statistics(mut self, statistics: Statistics) -> Self { + self.statistics = statistics; + self + } + + /// Set the projection of the files + pub fn with_projection(mut self, projection: Option>) -> Self { + self.projection = projection; + self + } + + /// Set the limit of the files + pub fn with_limit(mut self, limit: Option) -> Self { + self.limit = limit; + self + } + + /// Add a file as a single group + /// + /// See [Self::file_groups] for more information. + pub fn with_file(self, file: PartitionedFile) -> Self { + self.with_file_group(vec![file]) + } + + /// Add the file groups + /// + /// See [Self::file_groups] for more information. + pub fn with_file_groups( + mut self, + mut file_groups: Vec>, + ) -> Self { + self.file_groups.append(&mut file_groups); + self + } + + /// Add a new file group + /// + /// See [Self::file_groups] for more information + pub fn with_file_group(mut self, file_group: Vec) -> Self { + self.file_groups.push(file_group); + self + } + + /// Set the partitioning columns of the files + pub fn with_table_partition_cols(mut self, table_partition_cols: Vec) -> Self { + self.table_partition_cols = table_partition_cols; + self + } + + /// Set the output ordering of the files + pub fn with_output_ordering(mut self, output_ordering: Vec) -> Self { + self.output_ordering = output_ordering; + self + } + + /// Project the schema, constraints, and the statistics on the given column indices + pub fn project(&self) -> (SchemaRef, Constraints, Statistics, Vec) { + if self.projection.is_none() && self.table_partition_cols.is_empty() { + return ( + Arc::clone(&self.file_schema), + self.constraints.clone(), + self.statistics.clone(), + self.output_ordering.clone(), + ); + } + + let proj_indices = if let Some(proj) = &self.projection { + proj + } else { + let len = self.file_schema.fields().len() + self.table_partition_cols.len(); + &(0..len).collect::>() + }; + + let mut table_fields = vec![]; + let mut table_cols_stats = vec![]; + for idx in proj_indices { + if *idx < self.file_schema.fields().len() { + let field = self.file_schema.field(*idx); + table_fields.push(field.clone()); + table_cols_stats.push(self.statistics.column_statistics[*idx].clone()) + } else { + let partition_idx = idx - self.file_schema.fields().len(); + table_fields.push(self.table_partition_cols[partition_idx].to_owned()); + // TODO provide accurate stat for partition column (#1186) + table_cols_stats.push(ColumnStatistics::new_unknown()) + } + } + + let table_stats = Statistics { + num_rows: self.statistics.num_rows, + // TODO correct byte size? + total_byte_size: Precision::Absent, + column_statistics: table_cols_stats, + }; + + let projected_schema = Arc::new(Schema::new_with_metadata( + table_fields, + self.file_schema.metadata().clone(), + )); + + let projected_constraints = self + .constraints + .project(proj_indices) + .unwrap_or_else(Constraints::empty); + + let projected_output_ordering = + get_projected_output_ordering(self, &projected_schema); + + ( + projected_schema, + projected_constraints, + table_stats, + projected_output_ordering, + ) + } + + #[cfg_attr(not(feature = "avro"), allow(unused))] // Only used by avro + pub fn projected_file_column_names(&self) -> Option> { + self.projection.as_ref().map(|p| { + p.iter() + .filter(|col_idx| **col_idx < self.file_schema.fields().len()) + .map(|col_idx| self.file_schema.field(*col_idx).name()) + .cloned() + .collect() + }) + } + + /// Projects only file schema, ignoring partition columns + pub fn projected_file_schema(&self) -> SchemaRef { + let fields = self.file_column_projection_indices().map(|indices| { + indices + .iter() + .map(|col_idx| self.file_schema.field(*col_idx)) + .cloned() + .collect::>() + }); + + fields.map_or_else( + || Arc::clone(&self.file_schema), + |f| { + Arc::new(Schema::new_with_metadata( + f, + self.file_schema.metadata.clone(), + )) + }, + ) + } + + pub fn file_column_projection_indices(&self) -> Option> { + self.projection.as_ref().map(|p| { + p.iter() + .filter(|col_idx| **col_idx < self.file_schema.fields().len()) + .copied() + .collect() + }) + } + + /// Attempts to do a bin-packing on files into file groups, such that any two files + /// in a file group are ordered and non-overlapping with respect to their statistics. + /// It will produce the smallest number of file groups possible. + pub fn split_groups_by_statistics( + table_schema: &SchemaRef, + file_groups: &[Vec], + sort_order: &LexOrdering, + ) -> Result>> { + let flattened_files = file_groups.iter().flatten().collect::>(); + // First Fit: + // * Choose the first file group that a file can be placed into. + // * If it fits into no existing file groups, create a new one. + // + // By sorting files by min values and then applying first-fit bin packing, + // we can produce the smallest number of file groups such that + // files within a group are in order and non-overlapping. + // + // Source: Applied Combinatorics (Keller and Trotter), Chapter 6.8 + // https://www.appliedcombinatorics.org/book/s_posets_dilworth-intord.html + + if flattened_files.is_empty() { + return Ok(vec![]); + } + + let statistics = MinMaxStatistics::new_from_files( + sort_order, + table_schema, + None, + flattened_files.iter().copied(), + ) + .map_err(|e| { + e.context("construct min/max statistics for split_groups_by_statistics") + })?; + + let indices_sorted_by_min = statistics.min_values_sorted(); + let mut file_groups_indices: Vec> = vec![]; + + for (idx, min) in indices_sorted_by_min { + let file_group_to_insert = file_groups_indices.iter_mut().find(|group| { + // If our file is non-overlapping and comes _after_ the last file, + // it fits in this file group. + min > statistics.max( + *group + .last() + .expect("groups should be nonempty at construction"), + ) + }); + match file_group_to_insert { + Some(group) => group.push(idx), + None => file_groups_indices.push(vec![idx]), + } + } + + // Assemble indices back into groups of PartitionedFiles + Ok(file_groups_indices + .into_iter() + .map(|file_group_indices| { + file_group_indices + .into_iter() + .map(|idx| flattened_files[idx].clone()) + .collect() + }) + .collect()) + } +} + +impl Debug for FileScanConfig { + fn fmt(&self, f: &mut Formatter<'_>) -> FmtResult { + write!(f, "object_store_url={:?}, ", self.object_store_url)?; + + write!(f, "statistics={:?}, ", self.statistics)?; + + DisplayAs::fmt_as(self, DisplayFormatType::Verbose, f) + } +} + +impl DisplayAs for FileScanConfig { + fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> FmtResult { + let (schema, _, _, orderings) = self.project(); + + write!(f, "file_groups=")?; + FileGroupsDisplay(&self.file_groups).fmt_as(t, f)?; + + if !schema.fields().is_empty() { + write!(f, ", projection={}", ProjectSchemaDisplay(&schema))?; + } + + if let Some(limit) = self.limit { + write!(f, ", limit={limit}")?; + } + + display_orderings(f, &orderings)?; + + if !self.constraints.is_empty() { + write!(f, ", {}", self.constraints)?; + } + + Ok(()) + } +} + +/// A helper that projects partition columns into the file record batches. +/// +/// One interesting trick is the usage of a cache for the key buffers of the partition column +/// dictionaries. Indeed, the partition columns are constant, so the dictionaries that represent them +/// have all their keys equal to 0. This enables us to re-use the same "all-zero" buffer across batches, +/// which makes the space consumption of the partition columns O(batch_size) instead of O(record_count). +pub struct PartitionColumnProjector { + /// An Arrow buffer initialized to zeros that represents the key array of all partition + /// columns (partition columns are materialized by dictionary arrays with only one + /// value in the dictionary, thus all the keys are equal to zero). + key_buffer_cache: ZeroBufferGenerators, + /// Mapping between the indexes in the list of partition columns and the target + /// schema. Sorted by index in the target schema so that we can iterate on it to + /// insert the partition columns in the target record batch. + projected_partition_indexes: Vec<(usize, usize)>, + /// The schema of the table once the projection was applied. + projected_schema: SchemaRef, +} + +impl PartitionColumnProjector { + // Create a projector to insert the partitioning columns into batches read from files + // - `projected_schema`: the target schema with both file and partitioning columns + // - `table_partition_cols`: all the partitioning column names + pub fn new(projected_schema: SchemaRef, table_partition_cols: &[String]) -> Self { + let mut idx_map = HashMap::new(); + for (partition_idx, partition_name) in table_partition_cols.iter().enumerate() { + if let Ok(schema_idx) = projected_schema.index_of(partition_name) { + idx_map.insert(partition_idx, schema_idx); + } + } + + let mut projected_partition_indexes: Vec<_> = idx_map.into_iter().collect(); + projected_partition_indexes.sort_by(|(_, a), (_, b)| a.cmp(b)); + + Self { + projected_partition_indexes, + key_buffer_cache: Default::default(), + projected_schema, + } + } + + // Transform the batch read from the file by inserting the partitioning columns + // to the right positions as deduced from `projected_schema` + // - `file_batch`: batch read from the file, with internal projection applied + // - `partition_values`: the list of partition values, one for each partition column + pub fn project( + &mut self, + file_batch: RecordBatch, + partition_values: &[ScalarValue], + ) -> Result { + let expected_cols = + self.projected_schema.fields().len() - self.projected_partition_indexes.len(); + + if file_batch.columns().len() != expected_cols { + return exec_err!( + "Unexpected batch schema from file, expected {} cols but got {}", + expected_cols, + file_batch.columns().len() + ); + } + + let mut cols = file_batch.columns().to_vec(); + for &(pidx, sidx) in &self.projected_partition_indexes { + let p_value = + partition_values + .get(pidx) + .ok_or(DataFusionError::Execution( + "Invalid partitioning found on disk".to_string(), + ))?; + + let mut partition_value = Cow::Borrowed(p_value); + + // check if user forgot to dict-encode the partition value + let field = self.projected_schema.field(sidx); + let expected_data_type = field.data_type(); + let actual_data_type = partition_value.data_type(); + if let DataType::Dictionary(key_type, _) = expected_data_type { + if !matches!(actual_data_type, DataType::Dictionary(_, _)) { + warn!("Partition value for column {} was not dictionary-encoded, applied auto-fix.", field.name()); + partition_value = Cow::Owned(ScalarValue::Dictionary( + key_type.clone(), + Box::new(partition_value.as_ref().clone()), + )); + } + } + + cols.insert( + sidx, + create_output_array( + &mut self.key_buffer_cache, + partition_value.as_ref(), + file_batch.num_rows(), + )?, + ) + } + + RecordBatch::try_new_with_options( + Arc::clone(&self.projected_schema), + cols, + &RecordBatchOptions::new().with_row_count(Some(file_batch.num_rows())), + ) + .map_err(Into::into) + } +} + +#[derive(Debug, Default)] +struct ZeroBufferGenerators { + gen_i8: ZeroBufferGenerator, + gen_i16: ZeroBufferGenerator, + gen_i32: ZeroBufferGenerator, + gen_i64: ZeroBufferGenerator, + gen_u8: ZeroBufferGenerator, + gen_u16: ZeroBufferGenerator, + gen_u32: ZeroBufferGenerator, + gen_u64: ZeroBufferGenerator, +} + +/// Generate a arrow [`Buffer`] that contains zero values. +#[derive(Debug, Default)] +struct ZeroBufferGenerator +where + T: ArrowNativeType, +{ + cache: Option, + _t: PhantomData, +} + +impl ZeroBufferGenerator +where + T: ArrowNativeType, +{ + const SIZE: usize = size_of::(); + + fn get_buffer(&mut self, n_vals: usize) -> Buffer { + match &mut self.cache { + Some(buf) if buf.len() >= n_vals * Self::SIZE => { + buf.slice_with_length(0, n_vals * Self::SIZE) + } + _ => { + let mut key_buffer_builder = BufferBuilder::::new(n_vals); + key_buffer_builder.advance(n_vals); // keys are all 0 + self.cache.insert(key_buffer_builder.finish()).clone() + } + } + } +} + +fn create_dict_array( + buffer_gen: &mut ZeroBufferGenerator, + dict_val: &ScalarValue, + len: usize, + data_type: DataType, +) -> Result +where + T: ArrowNativeType, +{ + let dict_vals = dict_val.to_array()?; + + let sliced_key_buffer = buffer_gen.get_buffer(len); + + // assemble pieces together + let mut builder = ArrayData::builder(data_type) + .len(len) + .add_buffer(sliced_key_buffer); + builder = builder.add_child_data(dict_vals.to_data()); + Ok(Arc::new(DictionaryArray::::from( + builder.build().unwrap(), + ))) +} + +fn create_output_array( + key_buffer_cache: &mut ZeroBufferGenerators, + val: &ScalarValue, + len: usize, +) -> Result { + if let ScalarValue::Dictionary(key_type, dict_val) = &val { + match key_type.as_ref() { + DataType::Int8 => { + return create_dict_array( + &mut key_buffer_cache.gen_i8, + dict_val, + len, + val.data_type(), + ); + } + DataType::Int16 => { + return create_dict_array( + &mut key_buffer_cache.gen_i16, + dict_val, + len, + val.data_type(), + ); + } + DataType::Int32 => { + return create_dict_array( + &mut key_buffer_cache.gen_i32, + dict_val, + len, + val.data_type(), + ); + } + DataType::Int64 => { + return create_dict_array( + &mut key_buffer_cache.gen_i64, + dict_val, + len, + val.data_type(), + ); + } + DataType::UInt8 => { + return create_dict_array( + &mut key_buffer_cache.gen_u8, + dict_val, + len, + val.data_type(), + ); + } + DataType::UInt16 => { + return create_dict_array( + &mut key_buffer_cache.gen_u16, + dict_val, + len, + val.data_type(), + ); + } + DataType::UInt32 => { + return create_dict_array( + &mut key_buffer_cache.gen_u32, + dict_val, + len, + val.data_type(), + ); + } + DataType::UInt64 => { + return create_dict_array( + &mut key_buffer_cache.gen_u64, + dict_val, + len, + val.data_type(), + ); + } + _ => {} + } + } + + val.to_array_of_size(len) +} + +#[cfg(test)] +mod tests { + use arrow::array::Int32Array; + use arrow_schema::SortOptions; + use datafusion_common::assert_batches_eq; + use datafusion_expr::SortExpr; + use datafusion_physical_expr::create_physical_expr; + + use super::*; + + /// Get the schema for the aggregate_test_* csv files + fn aggr_test_schema() -> SchemaRef { + let mut f1 = Field::new("c1", DataType::Utf8, false); + f1.set_metadata(HashMap::from_iter(vec![("testing".into(), "test".into())])); + let schema = Schema::new(vec![ + f1, + Field::new("c2", DataType::UInt32, false), + Field::new("c3", DataType::Int8, false), + Field::new("c4", DataType::Int16, false), + Field::new("c5", DataType::Int32, false), + Field::new("c6", DataType::Int64, false), + Field::new("c7", DataType::UInt8, false), + Field::new("c8", DataType::UInt16, false), + Field::new("c9", DataType::UInt32, false), + Field::new("c10", DataType::UInt64, false), + Field::new("c11", DataType::Float32, false), + Field::new("c12", DataType::Float64, false), + Field::new("c13", DataType::Utf8, false), + ]); + + Arc::new(schema) + } + + /// Returns the column names on the schema + fn columns(schema: &Schema) -> Vec { + schema.fields().iter().map(|f| f.name().clone()).collect() + } + + #[test] + fn physical_plan_config_no_projection() { + let file_schema = aggr_test_schema(); + let conf = config_for_projection( + Arc::clone(&file_schema), + None, + Statistics::new_unknown(&file_schema), + to_partition_cols(vec![( + "date".to_owned(), + wrap_partition_type_in_dict(DataType::Utf8), + )]), + ); + + let (proj_schema, _, proj_statistics, _) = conf.project(); + assert_eq!(proj_schema.fields().len(), file_schema.fields().len() + 1); + assert_eq!( + proj_schema.field(file_schema.fields().len()).name(), + "date", + "partition columns are the last columns" + ); + assert_eq!( + proj_statistics.column_statistics.len(), + file_schema.fields().len() + 1 + ); + // TODO implement tests for partition column statistics once implemented + + let col_names = conf.projected_file_column_names(); + assert_eq!(col_names, None); + + let col_indices = conf.file_column_projection_indices(); + assert_eq!(col_indices, None); + } + + #[test] + fn physical_plan_config_no_projection_tab_cols_as_field() { + let file_schema = aggr_test_schema(); + + // make a table_partition_col as a field + let table_partition_col = + Field::new("date", wrap_partition_type_in_dict(DataType::Utf8), true) + .with_metadata(HashMap::from_iter(vec![( + "key_whatever".to_owned(), + "value_whatever".to_owned(), + )])); + + let conf = config_for_projection( + Arc::clone(&file_schema), + None, + Statistics::new_unknown(&file_schema), + vec![table_partition_col.clone()], + ); + + // verify the proj_schema includes the last column and exactly the same the field it is defined + let (proj_schema, _, _, _) = conf.project(); + assert_eq!(proj_schema.fields().len(), file_schema.fields().len() + 1); + assert_eq!( + *proj_schema.field(file_schema.fields().len()), + table_partition_col, + "partition columns are the last columns and ust have all values defined in created field" + ); + } + + #[test] + fn physical_plan_config_with_projection() { + let file_schema = aggr_test_schema(); + let conf = config_for_projection( + Arc::clone(&file_schema), + Some(vec![file_schema.fields().len(), 0]), + Statistics { + num_rows: Precision::Inexact(10), + // assign the column index to distinct_count to help assert + // the source statistic after the projection + column_statistics: (0..file_schema.fields().len()) + .map(|i| ColumnStatistics { + distinct_count: Precision::Inexact(i), + ..Default::default() + }) + .collect(), + total_byte_size: Precision::Absent, + }, + to_partition_cols(vec![( + "date".to_owned(), + wrap_partition_type_in_dict(DataType::Utf8), + )]), + ); + + let (proj_schema, _, proj_statistics, _) = conf.project(); + assert_eq!( + columns(&proj_schema), + vec!["date".to_owned(), "c1".to_owned()] + ); + let proj_stat_cols = proj_statistics.column_statistics; + assert_eq!(proj_stat_cols.len(), 2); + // TODO implement tests for proj_stat_cols[0] once partition column + // statistics are implemented + assert_eq!(proj_stat_cols[1].distinct_count, Precision::Inexact(0)); + + let col_names = conf.projected_file_column_names(); + assert_eq!(col_names, Some(vec!["c1".to_owned()])); + + let col_indices = conf.file_column_projection_indices(); + assert_eq!(col_indices, Some(vec![0])); + } + + #[test] + fn partition_column_projector() { + let file_batch = build_table_i32( + ("a", &vec![0, 1, 2]), + ("b", &vec![-2, -1, 0]), + ("c", &vec![10, 11, 12]), + ); + let partition_cols = vec![ + ( + "year".to_owned(), + wrap_partition_type_in_dict(DataType::Utf8), + ), + ( + "month".to_owned(), + wrap_partition_type_in_dict(DataType::Utf8), + ), + ( + "day".to_owned(), + wrap_partition_type_in_dict(DataType::Utf8), + ), + ]; + // create a projected schema + let conf = config_for_projection( + file_batch.schema(), + // keep all cols from file and 2 from partitioning + Some(vec![ + 0, + 1, + 2, + file_batch.schema().fields().len(), + file_batch.schema().fields().len() + 2, + ]), + Statistics::new_unknown(&file_batch.schema()), + to_partition_cols(partition_cols.clone()), + ); + let (proj_schema, ..) = conf.project(); + // created a projector for that projected schema + let mut proj = PartitionColumnProjector::new( + proj_schema, + &partition_cols + .iter() + .map(|x| x.0.clone()) + .collect::>(), + ); + + // project first batch + let projected_batch = proj + .project( + // file_batch is ok here because we kept all the file cols in the projection + file_batch, + &[ + wrap_partition_value_in_dict(ScalarValue::from("2021")), + wrap_partition_value_in_dict(ScalarValue::from("10")), + wrap_partition_value_in_dict(ScalarValue::from("26")), + ], + ) + .expect("Projection of partition columns into record batch failed"); + let expected = [ + "+---+----+----+------+-----+", + "| a | b | c | year | day |", + "+---+----+----+------+-----+", + "| 0 | -2 | 10 | 2021 | 26 |", + "| 1 | -1 | 11 | 2021 | 26 |", + "| 2 | 0 | 12 | 2021 | 26 |", + "+---+----+----+------+-----+", + ]; + assert_batches_eq!(expected, &[projected_batch]); + + // project another batch that is larger than the previous one + let file_batch = build_table_i32( + ("a", &vec![5, 6, 7, 8, 9]), + ("b", &vec![-10, -9, -8, -7, -6]), + ("c", &vec![12, 13, 14, 15, 16]), + ); + let projected_batch = proj + .project( + // file_batch is ok here because we kept all the file cols in the projection + file_batch, + &[ + wrap_partition_value_in_dict(ScalarValue::from("2021")), + wrap_partition_value_in_dict(ScalarValue::from("10")), + wrap_partition_value_in_dict(ScalarValue::from("27")), + ], + ) + .expect("Projection of partition columns into record batch failed"); + let expected = [ + "+---+-----+----+------+-----+", + "| a | b | c | year | day |", + "+---+-----+----+------+-----+", + "| 5 | -10 | 12 | 2021 | 27 |", + "| 6 | -9 | 13 | 2021 | 27 |", + "| 7 | -8 | 14 | 2021 | 27 |", + "| 8 | -7 | 15 | 2021 | 27 |", + "| 9 | -6 | 16 | 2021 | 27 |", + "+---+-----+----+------+-----+", + ]; + assert_batches_eq!(expected, &[projected_batch]); + + // project another batch that is smaller than the previous one + let file_batch = build_table_i32( + ("a", &vec![0, 1, 3]), + ("b", &vec![2, 3, 4]), + ("c", &vec![4, 5, 6]), + ); + let projected_batch = proj + .project( + // file_batch is ok here because we kept all the file cols in the projection + file_batch, + &[ + wrap_partition_value_in_dict(ScalarValue::from("2021")), + wrap_partition_value_in_dict(ScalarValue::from("10")), + wrap_partition_value_in_dict(ScalarValue::from("28")), + ], + ) + .expect("Projection of partition columns into record batch failed"); + let expected = [ + "+---+---+---+------+-----+", + "| a | b | c | year | day |", + "+---+---+---+------+-----+", + "| 0 | 2 | 4 | 2021 | 28 |", + "| 1 | 3 | 5 | 2021 | 28 |", + "| 3 | 4 | 6 | 2021 | 28 |", + "+---+---+---+------+-----+", + ]; + assert_batches_eq!(expected, &[projected_batch]); + + // forgot to dictionary-wrap the scalar value + let file_batch = build_table_i32( + ("a", &vec![0, 1, 2]), + ("b", &vec![-2, -1, 0]), + ("c", &vec![10, 11, 12]), + ); + let projected_batch = proj + .project( + // file_batch is ok here because we kept all the file cols in the projection + file_batch, + &[ + ScalarValue::from("2021"), + ScalarValue::from("10"), + ScalarValue::from("26"), + ], + ) + .expect("Projection of partition columns into record batch failed"); + let expected = [ + "+---+----+----+------+-----+", + "| a | b | c | year | day |", + "+---+----+----+------+-----+", + "| 0 | -2 | 10 | 2021 | 26 |", + "| 1 | -1 | 11 | 2021 | 26 |", + "| 2 | 0 | 12 | 2021 | 26 |", + "+---+----+----+------+-----+", + ]; + assert_batches_eq!(expected, &[projected_batch]); + } + + #[test] + fn test_projected_file_schema_with_partition_col() { + let schema = aggr_test_schema(); + let partition_cols = vec![ + ( + "part1".to_owned(), + wrap_partition_type_in_dict(DataType::Utf8), + ), + ( + "part2".to_owned(), + wrap_partition_type_in_dict(DataType::Utf8), + ), + ]; + + // Projected file schema for config with projection including partition column + let projection = config_for_projection( + schema.clone(), + Some(vec![0, 3, 5, schema.fields().len()]), + Statistics::new_unknown(&schema), + to_partition_cols(partition_cols), + ) + .projected_file_schema(); + + // Assert partition column filtered out in projected file schema + let expected_columns = vec!["c1", "c4", "c6"]; + let actual_columns = projection + .fields() + .iter() + .map(|f| f.name().clone()) + .collect::>(); + assert_eq!(expected_columns, actual_columns); + } + + #[test] + fn test_projected_file_schema_without_projection() { + let schema = aggr_test_schema(); + let partition_cols = vec![ + ( + "part1".to_owned(), + wrap_partition_type_in_dict(DataType::Utf8), + ), + ( + "part2".to_owned(), + wrap_partition_type_in_dict(DataType::Utf8), + ), + ]; + + // Projected file schema for config without projection + let projection = config_for_projection( + schema.clone(), + None, + Statistics::new_unknown(&schema), + to_partition_cols(partition_cols), + ) + .projected_file_schema(); + + // Assert projected file schema is equal to file schema + assert_eq!(projection.fields(), schema.fields()); + } + + #[test] + fn test_split_groups_by_statistics() -> Result<()> { + use chrono::TimeZone; + use datafusion_common::DFSchema; + use datafusion_expr::execution_props::ExecutionProps; + use object_store::{path::Path, ObjectMeta}; + + struct File { + name: &'static str, + date: &'static str, + statistics: Vec>, + } + impl File { + fn new( + name: &'static str, + date: &'static str, + statistics: Vec>, + ) -> Self { + Self { + name, + date, + statistics, + } + } + } + + struct TestCase { + name: &'static str, + file_schema: Schema, + files: Vec, + sort: Vec, + expected_result: Result>, &'static str>, + } + + use datafusion_expr::col; + let cases = vec![ + TestCase { + name: "test sort", + file_schema: Schema::new(vec![Field::new( + "value".to_string(), + DataType::Float64, + false, + )]), + files: vec![ + File::new("0", "2023-01-01", vec![Some((0.00, 0.49))]), + File::new("1", "2023-01-01", vec![Some((0.50, 1.00))]), + File::new("2", "2023-01-02", vec![Some((0.00, 1.00))]), + ], + sort: vec![col("value").sort(true, false)], + expected_result: Ok(vec![vec!["0", "1"], vec!["2"]]), + }, + // same input but file '2' is in the middle + // test that we still order correctly + TestCase { + name: "test sort with files ordered differently", + file_schema: Schema::new(vec![Field::new( + "value".to_string(), + DataType::Float64, + false, + )]), + files: vec![ + File::new("0", "2023-01-01", vec![Some((0.00, 0.49))]), + File::new("2", "2023-01-02", vec![Some((0.00, 1.00))]), + File::new("1", "2023-01-01", vec![Some((0.50, 1.00))]), + ], + sort: vec![col("value").sort(true, false)], + expected_result: Ok(vec![vec!["0", "1"], vec!["2"]]), + }, + TestCase { + name: "reverse sort", + file_schema: Schema::new(vec![Field::new( + "value".to_string(), + DataType::Float64, + false, + )]), + files: vec![ + File::new("0", "2023-01-01", vec![Some((0.00, 0.49))]), + File::new("1", "2023-01-01", vec![Some((0.50, 1.00))]), + File::new("2", "2023-01-02", vec![Some((0.00, 1.00))]), + ], + sort: vec![col("value").sort(false, true)], + expected_result: Ok(vec![vec!["1", "0"], vec!["2"]]), + }, + // reject nullable sort columns + TestCase { + name: "no nullable sort columns", + file_schema: Schema::new(vec![Field::new( + "value".to_string(), + DataType::Float64, + true, // should fail because nullable + )]), + files: vec![ + File::new("0", "2023-01-01", vec![Some((0.00, 0.49))]), + File::new("1", "2023-01-01", vec![Some((0.50, 1.00))]), + File::new("2", "2023-01-02", vec![Some((0.00, 1.00))]), + ], + sort: vec![col("value").sort(true, false)], + expected_result: Err("construct min/max statistics for split_groups_by_statistics\ncaused by\nbuild min rows\ncaused by\ncreate sorting columns\ncaused by\nError during planning: cannot sort by nullable column") + }, + TestCase { + name: "all three non-overlapping", + file_schema: Schema::new(vec![Field::new( + "value".to_string(), + DataType::Float64, + false, + )]), + files: vec![ + File::new("0", "2023-01-01", vec![Some((0.00, 0.49))]), + File::new("1", "2023-01-01", vec![Some((0.50, 0.99))]), + File::new("2", "2023-01-02", vec![Some((1.00, 1.49))]), + ], + sort: vec![col("value").sort(true, false)], + expected_result: Ok(vec![vec!["0", "1", "2"]]), + }, + TestCase { + name: "all three overlapping", + file_schema: Schema::new(vec![Field::new( + "value".to_string(), + DataType::Float64, + false, + )]), + files: vec![ + File::new("0", "2023-01-01", vec![Some((0.00, 0.49))]), + File::new("1", "2023-01-01", vec![Some((0.00, 0.49))]), + File::new("2", "2023-01-02", vec![Some((0.00, 0.49))]), + ], + sort: vec![col("value").sort(true, false)], + expected_result: Ok(vec![vec!["0"], vec!["1"], vec!["2"]]), + }, + TestCase { + name: "empty input", + file_schema: Schema::new(vec![Field::new( + "value".to_string(), + DataType::Float64, + false, + )]), + files: vec![], + sort: vec![col("value").sort(true, false)], + expected_result: Ok(vec![]), + }, + TestCase { + name: "one file missing statistics", + file_schema: Schema::new(vec![Field::new( + "value".to_string(), + DataType::Float64, + false, + )]), + files: vec![ + File::new("0", "2023-01-01", vec![Some((0.00, 0.49))]), + File::new("1", "2023-01-01", vec![Some((0.00, 0.49))]), + File::new("2", "2023-01-02", vec![None]), + ], + sort: vec![col("value").sort(true, false)], + expected_result: Err("construct min/max statistics for split_groups_by_statistics\ncaused by\ncollect min/max values\ncaused by\nget min/max for column: 'value'\ncaused by\nError during planning: statistics not found"), + }, + ]; + + for case in cases { + let table_schema = Arc::new(Schema::new( + case.file_schema + .fields() + .clone() + .into_iter() + .cloned() + .chain(Some(Arc::new(Field::new( + "date".to_string(), + DataType::Utf8, + false, + )))) + .collect::>(), + )); + + // This is a copy of datafusion::physical_planner::create_physical_sort_expr + // Create a physical sort expression from a logical expression + fn create_physical_sort_expr( + e: &SortExpr, + input_dfschema: &DFSchema, + execution_props: &ExecutionProps, + ) -> Result { + let SortExpr { + expr, + asc, + nulls_first, + } = e; + Ok(PhysicalSortExpr { + expr: create_physical_expr(expr, input_dfschema, execution_props)?, + options: SortOptions { + descending: !asc, + nulls_first: *nulls_first, + }, + }) + } + let sort_order = LexOrdering::from( + case.sort + .into_iter() + .map(|expr| { + create_physical_sort_expr( + &expr, + &DFSchema::try_from(table_schema.as_ref().clone())?, + &ExecutionProps::default(), + ) + }) + .collect::>>()?, + ); + + let partitioned_files = + case.files.into_iter().map(From::from).collect::>(); + let result = FileScanConfig::split_groups_by_statistics( + &table_schema, + &[partitioned_files.clone()], + &sort_order, + ); + let results_by_name = result + .as_ref() + .map(|file_groups| { + file_groups + .iter() + .map(|file_group| { + file_group + .iter() + .map(|file| { + partitioned_files + .iter() + .find_map(|f| { + if f.object_meta == file.object_meta { + Some( + f.object_meta + .location + .as_ref() + .rsplit('/') + .next() + .unwrap() + .trim_end_matches(".parquet"), + ) + } else { + None + } + }) + .unwrap() + }) + .collect::>() + }) + .collect::>() + }) + .map_err(|e| e.strip_backtrace().leak() as &'static str); + + assert_eq!(results_by_name, case.expected_result, "{}", case.name); + } + + return Ok(()); + + impl From for PartitionedFile { + fn from(file: File) -> Self { + PartitionedFile { + object_meta: ObjectMeta { + location: Path::from(format!( + "data/date={}/{}.parquet", + file.date, file.name + )), + last_modified: chrono::Utc.timestamp_nanos(0), + size: 0, + e_tag: None, + version: None, + }, + partition_values: vec![ScalarValue::from(file.date)], + range: None, + statistics: Some(Statistics { + num_rows: Precision::Absent, + total_byte_size: Precision::Absent, + column_statistics: file + .statistics + .into_iter() + .map(|stats| { + stats + .map(|(min, max)| ColumnStatistics { + min_value: Precision::Exact(ScalarValue::from( + min, + )), + max_value: Precision::Exact(ScalarValue::from( + max, + )), + ..Default::default() + }) + .unwrap_or_default() + }) + .collect::>(), + }), + extensions: None, + metadata_size_hint: None, + } + } + } + } + + // sets default for configs that play no role in projections + fn config_for_projection( + file_schema: SchemaRef, + projection: Option>, + statistics: Statistics, + table_partition_cols: Vec, + ) -> FileScanConfig { + FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), file_schema) + .with_projection(projection) + .with_statistics(statistics) + .with_table_partition_cols(table_partition_cols) + } + + /// Convert partition columns from Vec to Vec + fn to_partition_cols(table_partition_cols: Vec<(String, DataType)>) -> Vec { + table_partition_cols + .iter() + .map(|(name, dtype)| Field::new(name, dtype.clone(), false)) + .collect::>() + } + + /// returns record batch with 3 columns of i32 in memory + pub fn build_table_i32( + a: (&str, &Vec), + b: (&str, &Vec), + c: (&str, &Vec), + ) -> RecordBatch { + let schema = Schema::new(vec![ + Field::new(a.0, DataType::Int32, false), + Field::new(b.0, DataType::Int32, false), + Field::new(c.0, DataType::Int32, false), + ]); + + RecordBatch::try_new( + Arc::new(schema), + vec![ + Arc::new(Int32Array::from(a.1.clone())), + Arc::new(Int32Array::from(b.1.clone())), + Arc::new(Int32Array::from(c.1.clone())), + ], + ) + .unwrap() + } +} diff --git a/datafusion/catalog-listing/src/minmax_statistics.rs b/datafusion/catalog-listing/src/minmax_statistics.rs new file mode 100644 index 000000000000..71b2e551fd73 --- /dev/null +++ b/datafusion/catalog-listing/src/minmax_statistics.rs @@ -0,0 +1,287 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +/*! + * + * Use statistics to optimize physical planning. + * + * Currently, this module houses code to sort file groups if they are non-overlapping with + * respect to the required sort order. See [`MinMaxStatistics`] + * +*/ + +use std::sync::Arc; + +use crate::PartitionedFile; + +use arrow::array::RecordBatch; +use arrow::{ + compute::SortColumn, + row::{Row, Rows}, +}; +use arrow_schema::SchemaRef; +use datafusion_common::{plan_err, DataFusionError, Result}; +use datafusion_physical_expr::{expressions::Column, PhysicalSortExpr}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; + +/// A normalized representation of file min/max statistics that allows for efficient sorting & comparison. +/// The min/max values are ordered by [`Self::sort_order`]. +/// Furthermore, any columns that are reversed in the sort order have their min/max values swapped. +pub(crate) struct MinMaxStatistics { + min_by_sort_order: Rows, + max_by_sort_order: Rows, + sort_order: LexOrdering, +} + +impl MinMaxStatistics { + /// Sort order used to sort the statistics + #[allow(unused)] + pub fn sort_order(&self) -> &LexOrdering { + &self.sort_order + } + + /// Min value at index + #[allow(unused)] + pub fn min(&self, idx: usize) -> Row { + self.min_by_sort_order.row(idx) + } + + /// Max value at index + pub fn max(&self, idx: usize) -> Row { + self.max_by_sort_order.row(idx) + } + + pub fn new_from_files<'a>( + projected_sort_order: &LexOrdering, // Sort order with respect to projected schema + projected_schema: &SchemaRef, // Projected schema + projection: Option<&[usize]>, // Indices of projection in full table schema (None = all columns) + files: impl IntoIterator, + ) -> Result { + use datafusion_common::ScalarValue; + + let statistics_and_partition_values = files + .into_iter() + .map(|file| { + file.statistics + .as_ref() + .zip(Some(file.partition_values.as_slice())) + }) + .collect::>>() + .ok_or_else(|| { + DataFusionError::Plan("Parquet file missing statistics".to_string()) + })?; + + // Helper function to get min/max statistics for a given column of projected_schema + let get_min_max = |i: usize| -> Result<(Vec, Vec)> { + Ok(statistics_and_partition_values + .iter() + .map(|(s, pv)| { + if i < s.column_statistics.len() { + s.column_statistics[i] + .min_value + .get_value() + .cloned() + .zip(s.column_statistics[i].max_value.get_value().cloned()) + .ok_or_else(|| { + DataFusionError::Plan("statistics not found".to_string()) + }) + } else { + let partition_value = &pv[i - s.column_statistics.len()]; + Ok((partition_value.clone(), partition_value.clone())) + } + }) + .collect::>>()? + .into_iter() + .unzip()) + }; + + let sort_columns = sort_columns_from_physical_sort_exprs(projected_sort_order) + .ok_or(DataFusionError::Plan( + "sort expression must be on column".to_string(), + ))?; + + // Project the schema & sort order down to just the relevant columns + let min_max_schema = Arc::new( + projected_schema + .project(&(sort_columns.iter().map(|c| c.index()).collect::>()))?, + ); + let min_max_sort_order = LexOrdering::from( + sort_columns + .iter() + .zip(projected_sort_order.iter()) + .enumerate() + .map(|(i, (col, sort))| PhysicalSortExpr { + expr: Arc::new(Column::new(col.name(), i)), + options: sort.options, + }) + .collect::>(), + ); + + let (min_values, max_values): (Vec<_>, Vec<_>) = sort_columns + .iter() + .map(|c| { + // Reverse the projection to get the index of the column in the full statistics + // The file statistics contains _every_ column , but the sort column's index() + // refers to the index in projected_schema + let i = projection.map(|p| p[c.index()]).unwrap_or(c.index()); + + let (min, max) = get_min_max(i).map_err(|e| { + e.context(format!("get min/max for column: '{}'", c.name())) + })?; + Ok(( + ScalarValue::iter_to_array(min)?, + ScalarValue::iter_to_array(max)?, + )) + }) + .collect::>>() + .map_err(|e| e.context("collect min/max values"))? + .into_iter() + .unzip(); + + Self::new( + &min_max_sort_order, + &min_max_schema, + RecordBatch::try_new(Arc::clone(&min_max_schema), min_values).map_err( + |e| { + DataFusionError::ArrowError(e, Some("\ncreate min batch".to_string())) + }, + )?, + RecordBatch::try_new(Arc::clone(&min_max_schema), max_values).map_err( + |e| { + DataFusionError::ArrowError(e, Some("\ncreate max batch".to_string())) + }, + )?, + ) + } + + pub fn new( + sort_order: &LexOrdering, + schema: &SchemaRef, + min_values: RecordBatch, + max_values: RecordBatch, + ) -> Result { + use arrow::row::*; + + let sort_fields = sort_order + .iter() + .map(|expr| { + expr.expr + .data_type(schema) + .map(|data_type| SortField::new_with_options(data_type, expr.options)) + }) + .collect::>>() + .map_err(|e| e.context("create sort fields"))?; + let converter = RowConverter::new(sort_fields)?; + + let sort_columns = sort_columns_from_physical_sort_exprs(sort_order).ok_or( + DataFusionError::Plan("sort expression must be on column".to_string()), + )?; + + // swap min/max if they're reversed in the ordering + let (new_min_cols, new_max_cols): (Vec<_>, Vec<_>) = sort_order + .iter() + .zip(sort_columns.iter().copied()) + .map(|(sort_expr, column)| { + if sort_expr.options.descending { + max_values + .column_by_name(column.name()) + .zip(min_values.column_by_name(column.name())) + } else { + min_values + .column_by_name(column.name()) + .zip(max_values.column_by_name(column.name())) + } + .ok_or_else(|| { + DataFusionError::Plan(format!( + "missing column in MinMaxStatistics::new: '{}'", + column.name() + )) + }) + }) + .collect::>>()? + .into_iter() + .unzip(); + + let [min, max] = [new_min_cols, new_max_cols].map(|cols| { + let values = RecordBatch::try_new( + min_values.schema(), + cols.into_iter().cloned().collect(), + )?; + let sorting_columns = sort_order + .iter() + .zip(sort_columns.iter().copied()) + .map(|(sort_expr, column)| { + let schema = values.schema(); + + let idx = schema.index_of(column.name())?; + let field = schema.field(idx); + + // check that sort columns are non-nullable + if field.is_nullable() { + return plan_err!("cannot sort by nullable column"); + } + + Ok(SortColumn { + values: Arc::clone(values.column(idx)), + options: Some(sort_expr.options), + }) + }) + .collect::>>() + .map_err(|e| e.context("create sorting columns"))?; + converter + .convert_columns( + &sorting_columns + .into_iter() + .map(|c| c.values) + .collect::>(), + ) + .map_err(|e| { + DataFusionError::ArrowError(e, Some("convert columns".to_string())) + }) + }); + + Ok(Self { + min_by_sort_order: min.map_err(|e| e.context("build min rows"))?, + max_by_sort_order: max.map_err(|e| e.context("build max rows"))?, + sort_order: sort_order.clone(), + }) + } + + /// Return a sorted list of the min statistics together with the original indices + pub fn min_values_sorted(&self) -> Vec<(usize, Row<'_>)> { + let mut sort: Vec<_> = self.min_by_sort_order.iter().enumerate().collect(); + sort.sort_unstable_by(|(_, a), (_, b)| a.cmp(b)); + sort + } + + /// Check if the min/max statistics are in order and non-overlapping + pub fn is_sorted(&self) -> bool { + self.max_by_sort_order + .iter() + .zip(self.min_by_sort_order.iter().skip(1)) + .all(|(max, next_min)| max < next_min) + } +} + +fn sort_columns_from_physical_sort_exprs( + sort_order: &LexOrdering, +) -> Option> { + sort_order + .iter() + .map(|expr| expr.expr.as_any().downcast_ref::()) + .collect::>>() +} diff --git a/datafusion/core/src/datasource/file_format/arrow.rs b/datafusion/core/src/datasource/file_format/arrow.rs index 1d9827ae0ab5..d9c990d6937b 100644 --- a/datafusion/core/src/datasource/file_format/arrow.rs +++ b/datafusion/core/src/datasource/file_format/arrow.rs @@ -35,7 +35,6 @@ use crate::datasource::physical_plan::{ ArrowExec, FileGroupDisplay, FileScanConfig, FileSink, FileSinkConfig, }; use crate::error::Result; -use crate::execution::context::SessionState; use crate::physical_plan::{DisplayAs, DisplayFormatType, ExecutionPlan}; use arrow::ipc::convert::fb_to_schema; @@ -43,6 +42,7 @@ use arrow::ipc::reader::FileReader; use arrow::ipc::writer::IpcWriteOptions; use arrow::ipc::{root_as_message, CompressionType}; use arrow_schema::{ArrowError, Schema, SchemaRef}; +use datafusion_catalog::Session; use datafusion_common::parsers::CompressionTypeVariant; use datafusion_common::{ not_impl_err, DataFusionError, GetExt, Statistics, DEFAULT_ARROW_EXTENSION, @@ -83,7 +83,7 @@ impl ArrowFormatFactory { impl FileFormatFactory for ArrowFormatFactory { fn create( &self, - _state: &SessionState, + _state: &dyn Session, _format_options: &HashMap, ) -> Result> { Ok(Arc::new(ArrowFormat)) @@ -134,7 +134,7 @@ impl FileFormat for ArrowFormat { async fn infer_schema( &self, - _state: &SessionState, + _state: &dyn Session, store: &Arc, objects: &[ObjectMeta], ) -> Result { @@ -158,7 +158,7 @@ impl FileFormat for ArrowFormat { async fn infer_stats( &self, - _state: &SessionState, + _state: &dyn Session, _store: &Arc, table_schema: SchemaRef, _object: &ObjectMeta, @@ -168,7 +168,7 @@ impl FileFormat for ArrowFormat { async fn create_physical_plan( &self, - _state: &SessionState, + _state: &dyn Session, conf: FileScanConfig, _filters: Option<&Arc>, ) -> Result> { @@ -179,7 +179,7 @@ impl FileFormat for ArrowFormat { async fn create_writer_physical_plan( &self, input: Arc, - _state: &SessionState, + _state: &dyn Session, conf: FileSinkConfig, order_requirements: Option, ) -> Result> { diff --git a/datafusion/core/src/datasource/file_format/avro.rs b/datafusion/core/src/datasource/file_format/avro.rs index f854b9506a64..2bc25ca417f0 100644 --- a/datafusion/core/src/datasource/file_format/avro.rs +++ b/datafusion/core/src/datasource/file_format/avro.rs @@ -25,6 +25,7 @@ use std::sync::Arc; use arrow::datatypes::Schema; use arrow::datatypes::SchemaRef; use async_trait::async_trait; +use datafusion_catalog::Session; use datafusion_common::internal_err; use datafusion_common::parsers::CompressionTypeVariant; use datafusion_common::GetExt; @@ -38,7 +39,6 @@ use super::FileFormatFactory; use crate::datasource::avro_to_arrow::read_avro_schema_from_reader; use crate::datasource::physical_plan::{AvroExec, FileScanConfig}; use crate::error::Result; -use crate::execution::context::SessionState; use crate::physical_plan::ExecutionPlan; use crate::physical_plan::Statistics; @@ -56,7 +56,7 @@ impl AvroFormatFactory { impl FileFormatFactory for AvroFormatFactory { fn create( &self, - _state: &SessionState, + _state: &dyn Session, _format_options: &HashMap, ) -> Result> { Ok(Arc::new(AvroFormat)) @@ -111,7 +111,7 @@ impl FileFormat for AvroFormat { async fn infer_schema( &self, - _state: &SessionState, + _state: &dyn Session, store: &Arc, objects: &[ObjectMeta], ) -> Result { @@ -136,7 +136,7 @@ impl FileFormat for AvroFormat { async fn infer_stats( &self, - _state: &SessionState, + _state: &dyn Session, _store: &Arc, table_schema: SchemaRef, _object: &ObjectMeta, @@ -146,7 +146,7 @@ impl FileFormat for AvroFormat { async fn create_physical_plan( &self, - _state: &SessionState, + _state: &dyn Session, conf: FileScanConfig, _filters: Option<&Arc>, ) -> Result> { @@ -160,6 +160,7 @@ impl FileFormat for AvroFormat { mod tests { use super::*; use crate::datasource::file_format::test_util::scan_format; + use crate::execution::SessionState; use crate::physical_plan::collect; use crate::prelude::{SessionConfig, SessionContext}; use arrow::array::{as_string_array, Array}; diff --git a/datafusion/core/src/datasource/file_format/csv.rs b/datafusion/core/src/datasource/file_format/csv.rs index edf757e539a9..80b1e858c1c0 100644 --- a/datafusion/core/src/datasource/file_format/csv.rs +++ b/datafusion/core/src/datasource/file_format/csv.rs @@ -44,6 +44,7 @@ use arrow::array::RecordBatch; use arrow::csv::WriterBuilder; use arrow::datatypes::{DataType, Field, Fields, Schema, SchemaRef}; use arrow_schema::ArrowError; +use datafusion_catalog::Session; use datafusion_common::config::{ConfigField, ConfigFileType, CsvOptions}; use datafusion_common::file_options::csv_writer::CsvWriterOptions; use datafusion_common::{ @@ -94,9 +95,10 @@ impl Debug for CsvFormatFactory { impl FileFormatFactory for CsvFormatFactory { fn create( &self, - state: &SessionState, + state: &dyn Session, format_options: &HashMap, ) -> Result> { + let state = state.as_any().downcast_ref::().unwrap(); let csv_options = match &self.options { None => { let mut table_options = state.default_table_options(); @@ -364,7 +366,7 @@ impl FileFormat for CsvFormat { async fn infer_schema( &self, - state: &SessionState, + state: &dyn Session, store: &Arc, objects: &[ObjectMeta], ) -> Result { @@ -399,7 +401,7 @@ impl FileFormat for CsvFormat { async fn infer_stats( &self, - _state: &SessionState, + _state: &dyn Session, _store: &Arc, table_schema: SchemaRef, _object: &ObjectMeta, @@ -409,7 +411,7 @@ impl FileFormat for CsvFormat { async fn create_physical_plan( &self, - state: &SessionState, + state: &dyn Session, conf: FileScanConfig, _filters: Option<&Arc>, ) -> Result> { @@ -439,7 +441,7 @@ impl FileFormat for CsvFormat { async fn create_writer_physical_plan( &self, input: Arc, - state: &SessionState, + state: &dyn Session, conf: FileSinkConfig, order_requirements: Option, ) -> Result> { @@ -480,7 +482,7 @@ impl CsvFormat { /// number of lines that were read async fn infer_schema_from_stream( &self, - state: &SessionState, + state: &dyn Session, mut records_to_read: usize, stream: impl Stream>, ) -> Result<(Schema, usize)> { diff --git a/datafusion/core/src/datasource/file_format/json.rs b/datafusion/core/src/datasource/file_format/json.rs index 5bffb7e582c1..25adbb66c402 100644 --- a/datafusion/core/src/datasource/file_format/json.rs +++ b/datafusion/core/src/datasource/file_format/json.rs @@ -47,6 +47,7 @@ use arrow::json; use arrow::json::reader::{infer_json_schema_from_iterator, ValueIter}; use arrow_array::RecordBatch; use arrow_schema::ArrowError; +use datafusion_catalog::Session; use datafusion_common::config::{ConfigField, ConfigFileType, JsonOptions}; use datafusion_common::file_options::json_writer::JsonWriterOptions; use datafusion_common::{not_impl_err, GetExt, DEFAULT_JSON_EXTENSION}; @@ -85,9 +86,10 @@ impl JsonFormatFactory { impl FileFormatFactory for JsonFormatFactory { fn create( &self, - state: &SessionState, + state: &dyn Session, format_options: &HashMap, ) -> Result> { + let state = state.as_any().downcast_ref::().unwrap(); let json_options = match &self.options { None => { let mut table_options = state.default_table_options(); @@ -187,7 +189,7 @@ impl FileFormat for JsonFormat { async fn infer_schema( &self, - _state: &SessionState, + _state: &dyn Session, store: &Arc, objects: &[ObjectMeta], ) -> Result { @@ -235,7 +237,7 @@ impl FileFormat for JsonFormat { async fn infer_stats( &self, - _state: &SessionState, + _state: &dyn Session, _store: &Arc, table_schema: SchemaRef, _object: &ObjectMeta, @@ -245,7 +247,7 @@ impl FileFormat for JsonFormat { async fn create_physical_plan( &self, - _state: &SessionState, + _state: &dyn Session, conf: FileScanConfig, _filters: Option<&Arc>, ) -> Result> { @@ -257,7 +259,7 @@ impl FileFormat for JsonFormat { async fn create_writer_physical_plan( &self, input: Arc, - _state: &SessionState, + _state: &dyn Session, conf: FileSinkConfig, order_requirements: Option, ) -> Result> { diff --git a/datafusion/core/src/datasource/file_format/mod.rs b/datafusion/core/src/datasource/file_format/mod.rs index 2e2e6dba1c0e..ce94098501c0 100644 --- a/datafusion/core/src/datasource/file_format/mod.rs +++ b/datafusion/core/src/datasource/file_format/mod.rs @@ -39,12 +39,12 @@ use std::task::Poll; use crate::arrow::datatypes::SchemaRef; use crate::datasource::physical_plan::{FileScanConfig, FileSinkConfig}; -use crate::error::Result; -use crate::execution::context::SessionState; -use crate::physical_plan::{ExecutionPlan, Statistics}; +use datafusion_common::error::Result; +use datafusion_physical_plan::{ExecutionPlan, Statistics}; use arrow_array::RecordBatch; use arrow_schema::{ArrowError, DataType, Field, FieldRef, Schema}; +use datafusion_catalog::Session; use datafusion_common::file_options::file_type::FileType; use datafusion_common::{internal_err, not_impl_err, GetExt}; use datafusion_expr::Expr; @@ -65,7 +65,7 @@ pub trait FileFormatFactory: Sync + Send + GetExt + Debug { /// Initialize a [FileFormat] and configure based on session and command level options fn create( &self, - state: &SessionState, + state: &dyn Session, format_options: &HashMap, ) -> Result>; @@ -103,7 +103,7 @@ pub trait FileFormat: Send + Sync + Debug { /// the files have schemas that cannot be merged. async fn infer_schema( &self, - state: &SessionState, + state: &dyn Session, store: &Arc, objects: &[ObjectMeta], ) -> Result; @@ -117,7 +117,7 @@ pub trait FileFormat: Send + Sync + Debug { /// TODO: should the file source return statistics for only columns referred to in the table schema? async fn infer_stats( &self, - state: &SessionState, + state: &dyn Session, store: &Arc, table_schema: SchemaRef, object: &ObjectMeta, @@ -127,7 +127,7 @@ pub trait FileFormat: Send + Sync + Debug { /// according to this file format. async fn create_physical_plan( &self, - state: &SessionState, + state: &dyn Session, conf: FileScanConfig, filters: Option<&Arc>, ) -> Result>; @@ -137,7 +137,7 @@ pub trait FileFormat: Send + Sync + Debug { async fn create_writer_physical_plan( &self, _input: Arc, - _state: &SessionState, + _state: &dyn Session, _conf: FileSinkConfig, _order_requirements: Option, ) -> Result> { @@ -565,7 +565,7 @@ pub(crate) mod test_util { }; pub async fn scan_format( - state: &SessionState, + state: &dyn Session, format: &dyn FileFormat, store_root: &str, file_name: &str, diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index 4c7169764a76..a1978d6699d0 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -48,6 +48,7 @@ use crate::physical_plan::{ }; use arrow::compute::sum; +use datafusion_catalog::Session; use datafusion_common::config::{ConfigField, ConfigFileType, TableParquetOptions}; use datafusion_common::parsers::CompressionTypeVariant; use datafusion_common::stats::Precision; @@ -121,9 +122,10 @@ impl ParquetFormatFactory { impl FileFormatFactory for ParquetFormatFactory { fn create( &self, - state: &SessionState, + state: &dyn Session, format_options: &std::collections::HashMap, ) -> Result> { + let state = state.as_any().downcast_ref::().unwrap(); let parquet_options = match &self.options { None => { let mut table_options = state.default_table_options(); @@ -325,7 +327,7 @@ impl FileFormat for ParquetFormat { async fn infer_schema( &self, - state: &SessionState, + state: &dyn Session, store: &Arc, objects: &[ObjectMeta], ) -> Result { @@ -378,7 +380,7 @@ impl FileFormat for ParquetFormat { async fn infer_stats( &self, - _state: &SessionState, + _state: &dyn Session, store: &Arc, table_schema: SchemaRef, object: &ObjectMeta, @@ -395,7 +397,7 @@ impl FileFormat for ParquetFormat { async fn create_physical_plan( &self, - _state: &SessionState, + _state: &dyn Session, conf: FileScanConfig, filters: Option<&Arc>, ) -> Result> { @@ -420,7 +422,7 @@ impl FileFormat for ParquetFormat { async fn create_writer_physical_plan( &self, input: Arc, - _state: &SessionState, + _state: &dyn Session, conf: FileSinkConfig, order_requirements: Option, ) -> Result> { @@ -2216,13 +2218,13 @@ mod tests { } async fn get_exec( - state: &SessionState, + state: &dyn Session, file_name: &str, projection: Option>, limit: Option, ) -> Result> { let testdata = crate::test_util::parquet_test_data(); - + let state = state.as_any().downcast_ref::().unwrap(); let format = state .get_file_format_factory("parquet") .map(|factory| factory.create(state, &Default::default()).unwrap()) diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 9b9bcd22c464..0d011b29acc3 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -32,6 +32,7 @@ use crate::datasource::{ physical_plan::{FileScanConfig, FileSinkConfig}, }; use crate::execution::context::SessionState; + use datafusion_catalog::TableProvider; use datafusion_common::{config_err, DataFusionError, Result}; use datafusion_expr::dml::InsertOp; @@ -191,7 +192,7 @@ impl ListingTableConfig { } /// Infer the [`SchemaRef`] based on `table_path` suffix. Requires `self.options` to be set prior to using. - pub async fn infer_schema(self, state: &SessionState) -> Result { + pub async fn infer_schema(self, state: &dyn Session) -> Result { match self.options { Some(options) => { let schema = if let Some(url) = self.table_paths.first() { @@ -216,7 +217,7 @@ impl ListingTableConfig { } /// Infer the partition columns from the path. Requires `self.options` to be set prior to using. - pub async fn infer_partitions_from_path(self, state: &SessionState) -> Result { + pub async fn infer_partitions_from_path(self, state: &dyn Session) -> Result { match self.options { Some(options) => { let Some(url) = self.table_paths.first() else { @@ -484,7 +485,7 @@ impl ListingOptions { /// locally or ask a remote service to do it (e.g a scheduler). pub async fn infer_schema<'a>( &'a self, - state: &SessionState, + state: &dyn Session, table_path: &'a ListingTableUrl, ) -> Result { let store = state.runtime_env().object_store(table_path)?; @@ -509,7 +510,7 @@ impl ListingOptions { /// Allows specifying partial partitions. pub async fn validate_partitions( &self, - state: &SessionState, + state: &dyn Session, table_path: &ListingTableUrl, ) -> Result<()> { if self.table_partition_cols.is_empty() { @@ -563,7 +564,7 @@ impl ListingOptions { /// and therefore may fail to detect invalid partitioning. pub(crate) async fn infer_partitions( &self, - state: &SessionState, + state: &dyn Session, table_path: &ListingTableUrl, ) -> Result> { let store = state.runtime_env().object_store(table_path)?; @@ -1091,7 +1092,7 @@ impl ListingTable { /// be distributed to different threads / executors. async fn list_files_for_scan<'a>( &'a self, - ctx: &'a SessionState, + ctx: &'a dyn Session, filters: &'a [Expr], limit: Option, ) -> Result<(Vec>, Statistics)> { @@ -1152,7 +1153,7 @@ impl ListingTable { /// If they are not, it infers the statistics from the file and stores them in the cache. async fn do_collect_statistics( &self, - ctx: &SessionState, + ctx: &dyn Session, store: &Arc, part_file: &PartitionedFile, ) -> Result> { diff --git a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs index 5a38886bb16f..886492dbc231 100644 --- a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs +++ b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs @@ -24,8 +24,10 @@ use std::{ }; use super::{get_projected_output_ordering, statistics::MinMaxStatistics}; -use crate::datasource::{listing::PartitionedFile, object_store::ObjectStoreUrl}; -use crate::{error::Result, scalar::ScalarValue}; +use datafusion_catalog_listing::PartitionedFile; +use datafusion_execution::object_store::ObjectStoreUrl; +use datafusion_common::error::Result; +use datafusion_common::scalar::ScalarValue; use arrow::array::{ArrayData, BufferBuilder}; use arrow::buffer::Buffer; diff --git a/datafusion/core/src/datasource/physical_plan/file_stream.rs b/datafusion/core/src/datasource/physical_plan/file_stream.rs index 18cda4524ab2..7f95b909ee89 100644 --- a/datafusion/core/src/datasource/physical_plan/file_stream.rs +++ b/datafusion/core/src/datasource/physical_plan/file_stream.rs @@ -28,13 +28,13 @@ use std::sync::Arc; use std::task::{Context, Poll}; use crate::datasource::listing::PartitionedFile; -use crate::datasource::physical_plan::file_scan_config::PartitionColumnProjector; use crate::datasource::physical_plan::{FileMeta, FileScanConfig}; use crate::error::Result; use crate::physical_plan::metrics::{ BaselineMetrics, Count, ExecutionPlanMetricsSet, MetricBuilder, Time, }; use crate::physical_plan::RecordBatchStream; +use datafusion_catalog_listing::file_scan_config::PartitionColumnProjector; use arrow::datatypes::SchemaRef; use arrow::error::ArrowError; diff --git a/datafusion/core/src/datasource/physical_plan/mod.rs b/datafusion/core/src/datasource/physical_plan/mod.rs index 5bb7da8376a2..e5c823da0627 100644 --- a/datafusion/core/src/datasource/physical_plan/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/mod.rs @@ -21,12 +21,12 @@ mod arrow_file; mod avro; mod csv; mod file_groups; -mod file_scan_config; +// mod file_scan_config; mod file_stream; mod json; #[cfg(feature = "parquet")] pub mod parquet; -mod statistics; +// mod statistics; pub(crate) use self::csv::plan_to_csv; pub(crate) use self::json::plan_to_json; @@ -36,44 +36,33 @@ pub use self::parquet::{ParquetExec, ParquetFileMetrics, ParquetFileReaderFactor pub use arrow_file::ArrowExec; pub use avro::AvroExec; pub use csv::{CsvConfig, CsvExec, CsvExecBuilder, CsvOpener}; +pub use datafusion_catalog_listing::file_scan_config::{ + wrap_partition_type_in_dict, wrap_partition_value_in_dict, FileGroupDisplay, + FileScanConfig, +}; use datafusion_expr::dml::InsertOp; pub use file_groups::FileGroupPartitioner; -pub use file_scan_config::{ - wrap_partition_type_in_dict, wrap_partition_value_in_dict, FileScanConfig, -}; pub use file_stream::{FileOpenFuture, FileOpener, FileStream, OnError}; pub use json::{JsonOpener, NdJsonExec}; -use std::{ - fmt::{Debug, Formatter, Result as FmtResult}, - ops::Range, - sync::Arc, - vec, -}; +use std::{ops::Range, sync::Arc}; use super::{file_format::write::demux::start_demuxer_task, listing::ListingTableUrl}; use crate::datasource::file_format::write::demux::DemuxedStreamReceiver; -use crate::error::Result; -use crate::physical_plan::{DisplayAs, DisplayFormatType}; -use crate::{ - datasource::{ - listing::{FileRange, PartitionedFile}, - object_store::ObjectStoreUrl, - }, - physical_plan::display::{display_orderings, ProjectSchemaDisplay}, +use crate::datasource::{ + listing::{FileRange, PartitionedFile}, + object_store::ObjectStoreUrl, }; +use crate::error::Result; +use crate::physical_plan::DisplayAs; use arrow::datatypes::{DataType, SchemaRef}; use datafusion_common_runtime::SpawnedTask; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; -use datafusion_physical_expr::expressions::Column; -use datafusion_physical_expr::PhysicalSortExpr; -use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_plan::insert::DataSink; use async_trait::async_trait; use futures::StreamExt; -use log::debug; use object_store::{path::Path, GetOptions, GetRange, ObjectMeta, ObjectStore}; /// General behaviors for files that do `DataSink` operations @@ -161,151 +150,6 @@ impl FileSinkConfig { } } -impl Debug for FileScanConfig { - fn fmt(&self, f: &mut Formatter<'_>) -> FmtResult { - write!(f, "object_store_url={:?}, ", self.object_store_url)?; - - write!(f, "statistics={:?}, ", self.statistics)?; - - DisplayAs::fmt_as(self, DisplayFormatType::Verbose, f) - } -} - -impl DisplayAs for FileScanConfig { - fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> FmtResult { - let (schema, _, _, orderings) = self.project(); - - write!(f, "file_groups=")?; - FileGroupsDisplay(&self.file_groups).fmt_as(t, f)?; - - if !schema.fields().is_empty() { - write!(f, ", projection={}", ProjectSchemaDisplay(&schema))?; - } - - if let Some(limit) = self.limit { - write!(f, ", limit={limit}")?; - } - - display_orderings(f, &orderings)?; - - if !self.constraints.is_empty() { - write!(f, ", {}", self.constraints)?; - } - - Ok(()) - } -} - -/// A wrapper to customize partitioned file display -/// -/// Prints in the format: -/// ```text -/// {NUM_GROUPS groups: [[file1, file2,...], [fileN, fileM, ...], ...]} -/// ``` -#[derive(Debug)] -struct FileGroupsDisplay<'a>(&'a [Vec]); - -impl DisplayAs for FileGroupsDisplay<'_> { - fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> FmtResult { - let n_groups = self.0.len(); - let groups = if n_groups == 1 { "group" } else { "groups" }; - write!(f, "{{{n_groups} {groups}: [")?; - match t { - DisplayFormatType::Default => { - // To avoid showing too many partitions - let max_groups = 5; - fmt_up_to_n_elements(self.0, max_groups, f, |group, f| { - FileGroupDisplay(group).fmt_as(t, f) - })?; - } - DisplayFormatType::Verbose => { - fmt_elements_split_by_commas(self.0.iter(), f, |group, f| { - FileGroupDisplay(group).fmt_as(t, f) - })? - } - } - write!(f, "]}}") - } -} - -/// A wrapper to customize partitioned group of files display -/// -/// Prints in the format: -/// ```text -/// [file1, file2,...] -/// ``` -#[derive(Debug)] -pub(crate) struct FileGroupDisplay<'a>(pub &'a [PartitionedFile]); - -impl DisplayAs for FileGroupDisplay<'_> { - fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> FmtResult { - write!(f, "[")?; - match t { - DisplayFormatType::Default => { - // To avoid showing too many files - let max_files = 5; - fmt_up_to_n_elements(self.0, max_files, f, |pf, f| { - write!(f, "{}", pf.object_meta.location.as_ref())?; - if let Some(range) = pf.range.as_ref() { - write!(f, ":{}..{}", range.start, range.end)?; - } - Ok(()) - })? - } - DisplayFormatType::Verbose => { - fmt_elements_split_by_commas(self.0.iter(), f, |pf, f| { - write!(f, "{}", pf.object_meta.location.as_ref())?; - if let Some(range) = pf.range.as_ref() { - write!(f, ":{}..{}", range.start, range.end)?; - } - Ok(()) - })? - } - } - write!(f, "]") - } -} - -/// helper to format an array of up to N elements -fn fmt_up_to_n_elements( - elements: &[E], - n: usize, - f: &mut Formatter, - format_element: F, -) -> FmtResult -where - F: Fn(&E, &mut Formatter) -> FmtResult, -{ - let len = elements.len(); - fmt_elements_split_by_commas(elements.iter().take(n), f, |element, f| { - format_element(element, f) - })?; - // Remaining elements are showed as `...` (to indicate there is more) - if len > n { - write!(f, ", ...")?; - } - Ok(()) -} - -/// helper formatting array elements with a comma and a space between them -fn fmt_elements_split_by_commas( - iter: I, - f: &mut Formatter, - format_element: F, -) -> FmtResult -where - I: Iterator, - F: Fn(E, &mut Formatter) -> FmtResult, -{ - for (idx, element) in iter.enumerate() { - if idx > 0 { - write!(f, ", ")?; - } - format_element(element, f)?; - } - Ok(()) -} - /// A single file or part of a file that should be read, along with its schema, statistics pub struct FileMeta { /// Path for the file (e.g. URL, filesystem path, etc) @@ -336,131 +180,6 @@ impl From for FileMeta { } } -/// The various listing tables does not attempt to read all files -/// concurrently, instead they will read files in sequence within a -/// partition. This is an important property as it allows plans to -/// run against 1000s of files and not try to open them all -/// concurrently. -/// -/// However, it means if we assign more than one file to a partition -/// the output sort order will not be preserved as illustrated in the -/// following diagrams: -/// -/// When only 1 file is assigned to each partition, each partition is -/// correctly sorted on `(A, B, C)` -/// -/// ```text -///┏ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ┓ -/// ┌ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┐ ┌ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┌ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┌ ─ ─ ─ ─ ─ ─ ─ ─ ┐ -///┃ ┌───────────────┐ ┌──────────────┐ │ ┌──────────────┐ │ ┌─────────────┐ ┃ -/// │ │ 1.parquet │ │ │ │ 2.parquet │ │ │ 3.parquet │ │ │ 4.parquet │ │ -///┃ │ Sort: A, B, C │ │Sort: A, B, C │ │ │Sort: A, B, C │ │ │Sort: A, B, C│ ┃ -/// │ └───────────────┘ │ │ └──────────────┘ │ └──────────────┘ │ └─────────────┘ │ -///┃ │ │ ┃ -/// │ │ │ │ │ │ -///┃ │ │ ┃ -/// │ │ │ │ │ │ -///┃ │ │ ┃ -/// │ │ │ │ │ │ -///┃ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┘ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┘ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┃ -/// DataFusion DataFusion DataFusion DataFusion -///┃ Partition 1 Partition 2 Partition 3 Partition 4 ┃ -/// ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ -/// -/// ParquetExec -///``` -/// -/// However, when more than 1 file is assigned to each partition, each -/// partition is NOT correctly sorted on `(A, B, C)`. Once the second -/// file is scanned, the same values for A, B and C can be repeated in -/// the same sorted stream -/// -///```text -///┏ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ -/// ┌ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┐ ┌ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┃ -///┃ ┌───────────────┐ ┌──────────────┐ │ -/// │ │ 1.parquet │ │ │ │ 2.parquet │ ┃ -///┃ │ Sort: A, B, C │ │Sort: A, B, C │ │ -/// │ └───────────────┘ │ │ └──────────────┘ ┃ -///┃ ┌───────────────┐ ┌──────────────┐ │ -/// │ │ 3.parquet │ │ │ │ 4.parquet │ ┃ -///┃ │ Sort: A, B, C │ │Sort: A, B, C │ │ -/// │ └───────────────┘ │ │ └──────────────┘ ┃ -///┃ │ -/// │ │ │ ┃ -///┃ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┘ -/// DataFusion DataFusion ┃ -///┃ Partition 1 Partition 2 -/// ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ┛ -/// -/// ParquetExec -///``` -fn get_projected_output_ordering( - base_config: &FileScanConfig, - projected_schema: &SchemaRef, -) -> Vec { - let mut all_orderings = vec![]; - for output_ordering in &base_config.output_ordering { - let mut new_ordering = LexOrdering::default(); - for PhysicalSortExpr { expr, options } in output_ordering.iter() { - if let Some(col) = expr.as_any().downcast_ref::() { - let name = col.name(); - if let Some((idx, _)) = projected_schema.column_with_name(name) { - // Compute the new sort expression (with correct index) after projection: - new_ordering.push(PhysicalSortExpr { - expr: Arc::new(Column::new(name, idx)), - options: *options, - }); - continue; - } - } - // Cannot find expression in the projected_schema, stop iterating - // since rest of the orderings are violated - break; - } - - // do not push empty entries - // otherwise we may have `Some(vec![])` at the output ordering. - if new_ordering.is_empty() { - continue; - } - - // Check if any file groups are not sorted - if base_config.file_groups.iter().any(|group| { - if group.len() <= 1 { - // File groups with <= 1 files are always sorted - return false; - } - - let statistics = match statistics::MinMaxStatistics::new_from_files( - &new_ordering, - projected_schema, - base_config.projection.as_deref(), - group, - ) { - Ok(statistics) => statistics, - Err(e) => { - log::trace!("Error fetching statistics for file group: {e}"); - // we can't prove that it's ordered, so we have to reject it - return true; - } - }; - - !statistics.is_sorted() - }) { - debug!( - "Skipping specified output ordering {:?}. \ - Some file groups couldn't be determined to be sorted: {:?}", - base_config.output_ordering[0], base_config.file_groups - ); - continue; - } - - all_orderings.push(new_ordering); - } - all_orderings -} - /// Represents the possible outcomes of a range calculation. /// /// This enum is used to encapsulate the result of calculating the range of @@ -577,7 +296,7 @@ mod tests { StringArray, UInt64Array, }; use arrow_schema::{Field, Schema}; - + use crate::datasource::physical_plan::FileGroupDisplay; use crate::datasource::schema_adapter::{ DefaultSchemaAdapterFactory, SchemaAdapterFactory, }; diff --git a/datafusion/core/src/datasource/physical_plan/statistics.rs b/datafusion/core/src/datasource/physical_plan/statistics.rs index b4a8f377d256..b90c63537b3a 100644 --- a/datafusion/core/src/datasource/physical_plan/statistics.rs +++ b/datafusion/core/src/datasource/physical_plan/statistics.rs @@ -26,7 +26,7 @@ use std::sync::Arc; -use crate::datasource::listing::PartitionedFile; +use datafusion_catalog_listing::PartitionedFile; use arrow::{ compute::SortColumn, diff --git a/datafusion/core/tests/catalog_listing/helpers.rs b/datafusion/core/tests/catalog_listing/helpers.rs new file mode 100644 index 000000000000..6bd0feae70ed --- /dev/null +++ b/datafusion/core/tests/catalog_listing/helpers.rs @@ -0,0 +1,224 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use arrow::datatypes::DataType; +use datafusion::test::object_store::make_test_store_and_state; +use datafusion_catalog_listing::helpers::*; +use datafusion_catalog_listing::*; +use datafusion_common::ScalarValue; +use datafusion_expr::{col, lit, Expr}; +use futures::StreamExt; +use futures::TryStreamExt; + +#[tokio::test] +async fn test_pruned_partition_list_empty() { + let (store, state) = make_test_store_and_state(&[ + ("tablepath/mypartition=val1/notparquetfile", 100), + ("tablepath/mypartition=val1/ignoresemptyfile.parquet", 0), + ("tablepath/file.parquet", 100), + ]); + let filter = Expr::eq(col("mypartition"), lit("val1")); + let pruned = pruned_partition_list( + &state, + store.as_ref(), + &ListingTableUrl::parse("file:///tablepath/").unwrap(), + &[filter], + ".parquet", + &[(String::from("mypartition"), DataType::Utf8)], + ) + .await + .expect("partition pruning failed") + .collect::>() + .await; + + assert_eq!(pruned.len(), 0); +} + +#[tokio::test] +async fn test_pruned_partition_list() { + let (store, state) = make_test_store_and_state(&[ + ("tablepath/mypartition=val1/file.parquet", 100), + ("tablepath/mypartition=val2/file.parquet", 100), + ("tablepath/mypartition=val1/ignoresemptyfile.parquet", 0), + ("tablepath/mypartition=val1/other=val3/file.parquet", 100), + ]); + let filter = Expr::eq(col("mypartition"), lit("val1")); + let pruned = pruned_partition_list( + &state, + store.as_ref(), + &ListingTableUrl::parse("file:///tablepath/").unwrap(), + &[filter], + ".parquet", + &[(String::from("mypartition"), DataType::Utf8)], + ) + .await + .expect("partition pruning failed") + .try_collect::>() + .await + .unwrap(); + + assert_eq!(pruned.len(), 2); + let f1 = &pruned[0]; + assert_eq!( + f1.object_meta.location.as_ref(), + "tablepath/mypartition=val1/file.parquet" + ); + assert_eq!(&f1.partition_values, &[ScalarValue::from("val1")]); + let f2 = &pruned[1]; + assert_eq!( + f2.object_meta.location.as_ref(), + "tablepath/mypartition=val1/other=val3/file.parquet" + ); + assert_eq!(f2.partition_values, &[ScalarValue::from("val1"),]); +} + +#[tokio::test] +async fn test_pruned_partition_list_multi() { + let (store, state) = make_test_store_and_state(&[ + ("tablepath/part1=p1v1/file.parquet", 100), + ("tablepath/part1=p1v2/part2=p2v1/file1.parquet", 100), + ("tablepath/part1=p1v2/part2=p2v1/file2.parquet", 100), + ("tablepath/part1=p1v3/part2=p2v1/file2.parquet", 100), + ("tablepath/part1=p1v2/part2=p2v2/file2.parquet", 100), + ]); + let filter1 = Expr::eq(col("part1"), lit("p1v2")); + let filter2 = Expr::eq(col("part2"), lit("p2v1")); + let pruned = pruned_partition_list( + &state, + store.as_ref(), + &ListingTableUrl::parse("file:///tablepath/").unwrap(), + &[filter1, filter2], + ".parquet", + &[ + (String::from("part1"), DataType::Utf8), + (String::from("part2"), DataType::Utf8), + ], + ) + .await + .expect("partition pruning failed") + .try_collect::>() + .await + .unwrap(); + + assert_eq!(pruned.len(), 2); + let f1 = &pruned[0]; + assert_eq!( + f1.object_meta.location.as_ref(), + "tablepath/part1=p1v2/part2=p2v1/file1.parquet" + ); + assert_eq!( + &f1.partition_values, + &[ScalarValue::from("p1v2"), ScalarValue::from("p2v1"),] + ); + let f2 = &pruned[1]; + assert_eq!( + f2.object_meta.location.as_ref(), + "tablepath/part1=p1v2/part2=p2v1/file2.parquet" + ); + assert_eq!( + &f2.partition_values, + &[ScalarValue::from("p1v2"), ScalarValue::from("p2v1")] + ); +} + +#[tokio::test] +async fn test_list_partition() { + let (store, _) = make_test_store_and_state(&[ + ("tablepath/part1=p1v1/file.parquet", 100), + ("tablepath/part1=p1v2/part2=p2v1/file1.parquet", 100), + ("tablepath/part1=p1v2/part2=p2v1/file2.parquet", 100), + ("tablepath/part1=p1v3/part2=p2v1/file3.parquet", 100), + ("tablepath/part1=p1v2/part2=p2v2/file4.parquet", 100), + ("tablepath/part1=p1v2/part2=p2v2/empty.parquet", 0), + ]); + + let partitions = list_partitions( + store.as_ref(), + &ListingTableUrl::parse("file:///tablepath/").unwrap(), + 0, + None, + ) + .await + .expect("listing partitions failed"); + + assert_eq!( + &partitions + .iter() + .map(describe_partition) + .collect::>(), + &vec![ + ("tablepath", 0, vec![]), + ("tablepath/part1=p1v1", 1, vec![]), + ("tablepath/part1=p1v2", 1, vec![]), + ("tablepath/part1=p1v3", 1, vec![]), + ] + ); + + let partitions = list_partitions( + store.as_ref(), + &ListingTableUrl::parse("file:///tablepath/").unwrap(), + 1, + None, + ) + .await + .expect("listing partitions failed"); + + assert_eq!( + &partitions + .iter() + .map(describe_partition) + .collect::>(), + &vec![ + ("tablepath", 0, vec![]), + ("tablepath/part1=p1v1", 1, vec!["file.parquet"]), + ("tablepath/part1=p1v2", 1, vec![]), + ("tablepath/part1=p1v2/part2=p2v1", 2, vec![]), + ("tablepath/part1=p1v2/part2=p2v2", 2, vec![]), + ("tablepath/part1=p1v3", 1, vec![]), + ("tablepath/part1=p1v3/part2=p2v1", 2, vec![]), + ] + ); + + let partitions = list_partitions( + store.as_ref(), + &ListingTableUrl::parse("file:///tablepath/").unwrap(), + 2, + None, + ) + .await + .expect("listing partitions failed"); + + assert_eq!( + &partitions + .iter() + .map(describe_partition) + .collect::>(), + &vec![ + ("tablepath", 0, vec![]), + ("tablepath/part1=p1v1", 1, vec!["file.parquet"]), + ("tablepath/part1=p1v2", 1, vec![]), + ("tablepath/part1=p1v3", 1, vec![]), + ( + "tablepath/part1=p1v2/part2=p2v1", + 2, + vec!["file1.parquet", "file2.parquet"] + ), + ("tablepath/part1=p1v2/part2=p2v2", 2, vec!["file4.parquet"]), + ("tablepath/part1=p1v3/part2=p2v1", 2, vec!["file3.parquet"]), + ] + ); +} diff --git a/datafusion/core/tests/catalog_listing/mod.rs b/datafusion/core/tests/catalog_listing/mod.rs new file mode 100644 index 000000000000..9df4cc1fa3ac --- /dev/null +++ b/datafusion/core/tests/catalog_listing/mod.rs @@ -0,0 +1,18 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +mod helpers; diff --git a/datafusion/core/tests/core_integration.rs b/datafusion/core/tests/core_integration.rs index 66b4103160e7..fb182baf1f87 100644 --- a/datafusion/core/tests/core_integration.rs +++ b/datafusion/core/tests/core_integration.rs @@ -46,6 +46,8 @@ mod physical_optimizer; mod catalog; +mod catalog_listing; + #[cfg(test)] #[ctor::ctor] fn init() { From 5c02d7e53224b3191713574fd7cc0fb5b9ba6392 Mon Sep 17 00:00:00 2001 From: logan-keede Date: Wed, 5 Feb 2025 23:39:50 +0530 Subject: [PATCH 2/7] mod fixe --- datafusion/catalog-listing/src/mod.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/datafusion/catalog-listing/src/mod.rs b/datafusion/catalog-listing/src/mod.rs index e952e39fd479..12d6cb532c6e 100644 --- a/datafusion/catalog-listing/src/mod.rs +++ b/datafusion/catalog-listing/src/mod.rs @@ -20,6 +20,8 @@ pub mod helpers; pub mod url; +pub mod file_scan_config; +mod minmax_statistics; use chrono::TimeZone; use datafusion_common::Result; From a1d26cabd327e7d265a522398cf32babfa884725 Mon Sep 17 00:00:00 2001 From: logan-keede Date: Wed, 5 Feb 2025 23:44:56 +0530 Subject: [PATCH 3/7] fix: Cargo fmt --- datafusion/catalog-listing/src/mod.rs | 4 ++-- datafusion/core/src/datasource/physical_plan/mod.rs | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/datafusion/catalog-listing/src/mod.rs b/datafusion/catalog-listing/src/mod.rs index 12d6cb532c6e..0cbdf5fc8376 100644 --- a/datafusion/catalog-listing/src/mod.rs +++ b/datafusion/catalog-listing/src/mod.rs @@ -18,10 +18,10 @@ //! A table that uses the `ObjectStore` listing capability //! to get the list of files to process. -pub mod helpers; -pub mod url; pub mod file_scan_config; +pub mod helpers; mod minmax_statistics; +pub mod url; use chrono::TimeZone; use datafusion_common::Result; diff --git a/datafusion/core/src/datasource/physical_plan/mod.rs b/datafusion/core/src/datasource/physical_plan/mod.rs index e5c823da0627..6370457cc364 100644 --- a/datafusion/core/src/datasource/physical_plan/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/mod.rs @@ -289,6 +289,10 @@ mod tests { use super::*; use crate::physical_plan::{DefaultDisplay, VerboseDisplay}; + use crate::datasource::physical_plan::FileGroupDisplay; + use crate::datasource::schema_adapter::{ + DefaultSchemaAdapterFactory, SchemaAdapterFactory, + }; use arrow_array::cast::AsArray; use arrow_array::types::{Float32Type, Float64Type, UInt32Type}; use arrow_array::{ @@ -296,10 +300,6 @@ mod tests { StringArray, UInt64Array, }; use arrow_schema::{Field, Schema}; - use crate::datasource::physical_plan::FileGroupDisplay; - use crate::datasource::schema_adapter::{ - DefaultSchemaAdapterFactory, SchemaAdapterFactory, - }; use chrono::Utc; #[test] From 0d6350783f81e1a2c641dac43e1cc04445431f7e Mon Sep 17 00:00:00 2001 From: logan-keede Date: Wed, 5 Feb 2025 23:52:58 +0530 Subject: [PATCH 4/7] removing remanant of previous commmit(bad merge) --- datafusion/catalog-listing/Cargo.toml | 1 + .../core/tests/catalog_listing/helpers.rs | 224 ------------------ datafusion/core/tests/catalog_listing/mod.rs | 18 -- 3 files changed, 1 insertion(+), 242 deletions(-) delete mode 100644 datafusion/core/tests/catalog_listing/helpers.rs delete mode 100644 datafusion/core/tests/catalog_listing/mod.rs diff --git a/datafusion/catalog-listing/Cargo.toml b/datafusion/catalog-listing/Cargo.toml index a0bb3de255e0..72402df8a2ce 100644 --- a/datafusion/catalog-listing/Cargo.toml +++ b/datafusion/catalog-listing/Cargo.toml @@ -59,6 +59,7 @@ url = { workspace = true } avro = ["apache-avro", "num-traits", "datafusion-common/avro"] [dev-dependencies] +async-trait = { workspace = true } tempfile = { workspace = true } tokio = { workspace = true } diff --git a/datafusion/core/tests/catalog_listing/helpers.rs b/datafusion/core/tests/catalog_listing/helpers.rs deleted file mode 100644 index 6bd0feae70ed..000000000000 --- a/datafusion/core/tests/catalog_listing/helpers.rs +++ /dev/null @@ -1,224 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -use arrow::datatypes::DataType; -use datafusion::test::object_store::make_test_store_and_state; -use datafusion_catalog_listing::helpers::*; -use datafusion_catalog_listing::*; -use datafusion_common::ScalarValue; -use datafusion_expr::{col, lit, Expr}; -use futures::StreamExt; -use futures::TryStreamExt; - -#[tokio::test] -async fn test_pruned_partition_list_empty() { - let (store, state) = make_test_store_and_state(&[ - ("tablepath/mypartition=val1/notparquetfile", 100), - ("tablepath/mypartition=val1/ignoresemptyfile.parquet", 0), - ("tablepath/file.parquet", 100), - ]); - let filter = Expr::eq(col("mypartition"), lit("val1")); - let pruned = pruned_partition_list( - &state, - store.as_ref(), - &ListingTableUrl::parse("file:///tablepath/").unwrap(), - &[filter], - ".parquet", - &[(String::from("mypartition"), DataType::Utf8)], - ) - .await - .expect("partition pruning failed") - .collect::>() - .await; - - assert_eq!(pruned.len(), 0); -} - -#[tokio::test] -async fn test_pruned_partition_list() { - let (store, state) = make_test_store_and_state(&[ - ("tablepath/mypartition=val1/file.parquet", 100), - ("tablepath/mypartition=val2/file.parquet", 100), - ("tablepath/mypartition=val1/ignoresemptyfile.parquet", 0), - ("tablepath/mypartition=val1/other=val3/file.parquet", 100), - ]); - let filter = Expr::eq(col("mypartition"), lit("val1")); - let pruned = pruned_partition_list( - &state, - store.as_ref(), - &ListingTableUrl::parse("file:///tablepath/").unwrap(), - &[filter], - ".parquet", - &[(String::from("mypartition"), DataType::Utf8)], - ) - .await - .expect("partition pruning failed") - .try_collect::>() - .await - .unwrap(); - - assert_eq!(pruned.len(), 2); - let f1 = &pruned[0]; - assert_eq!( - f1.object_meta.location.as_ref(), - "tablepath/mypartition=val1/file.parquet" - ); - assert_eq!(&f1.partition_values, &[ScalarValue::from("val1")]); - let f2 = &pruned[1]; - assert_eq!( - f2.object_meta.location.as_ref(), - "tablepath/mypartition=val1/other=val3/file.parquet" - ); - assert_eq!(f2.partition_values, &[ScalarValue::from("val1"),]); -} - -#[tokio::test] -async fn test_pruned_partition_list_multi() { - let (store, state) = make_test_store_and_state(&[ - ("tablepath/part1=p1v1/file.parquet", 100), - ("tablepath/part1=p1v2/part2=p2v1/file1.parquet", 100), - ("tablepath/part1=p1v2/part2=p2v1/file2.parquet", 100), - ("tablepath/part1=p1v3/part2=p2v1/file2.parquet", 100), - ("tablepath/part1=p1v2/part2=p2v2/file2.parquet", 100), - ]); - let filter1 = Expr::eq(col("part1"), lit("p1v2")); - let filter2 = Expr::eq(col("part2"), lit("p2v1")); - let pruned = pruned_partition_list( - &state, - store.as_ref(), - &ListingTableUrl::parse("file:///tablepath/").unwrap(), - &[filter1, filter2], - ".parquet", - &[ - (String::from("part1"), DataType::Utf8), - (String::from("part2"), DataType::Utf8), - ], - ) - .await - .expect("partition pruning failed") - .try_collect::>() - .await - .unwrap(); - - assert_eq!(pruned.len(), 2); - let f1 = &pruned[0]; - assert_eq!( - f1.object_meta.location.as_ref(), - "tablepath/part1=p1v2/part2=p2v1/file1.parquet" - ); - assert_eq!( - &f1.partition_values, - &[ScalarValue::from("p1v2"), ScalarValue::from("p2v1"),] - ); - let f2 = &pruned[1]; - assert_eq!( - f2.object_meta.location.as_ref(), - "tablepath/part1=p1v2/part2=p2v1/file2.parquet" - ); - assert_eq!( - &f2.partition_values, - &[ScalarValue::from("p1v2"), ScalarValue::from("p2v1")] - ); -} - -#[tokio::test] -async fn test_list_partition() { - let (store, _) = make_test_store_and_state(&[ - ("tablepath/part1=p1v1/file.parquet", 100), - ("tablepath/part1=p1v2/part2=p2v1/file1.parquet", 100), - ("tablepath/part1=p1v2/part2=p2v1/file2.parquet", 100), - ("tablepath/part1=p1v3/part2=p2v1/file3.parquet", 100), - ("tablepath/part1=p1v2/part2=p2v2/file4.parquet", 100), - ("tablepath/part1=p1v2/part2=p2v2/empty.parquet", 0), - ]); - - let partitions = list_partitions( - store.as_ref(), - &ListingTableUrl::parse("file:///tablepath/").unwrap(), - 0, - None, - ) - .await - .expect("listing partitions failed"); - - assert_eq!( - &partitions - .iter() - .map(describe_partition) - .collect::>(), - &vec![ - ("tablepath", 0, vec![]), - ("tablepath/part1=p1v1", 1, vec![]), - ("tablepath/part1=p1v2", 1, vec![]), - ("tablepath/part1=p1v3", 1, vec![]), - ] - ); - - let partitions = list_partitions( - store.as_ref(), - &ListingTableUrl::parse("file:///tablepath/").unwrap(), - 1, - None, - ) - .await - .expect("listing partitions failed"); - - assert_eq!( - &partitions - .iter() - .map(describe_partition) - .collect::>(), - &vec![ - ("tablepath", 0, vec![]), - ("tablepath/part1=p1v1", 1, vec!["file.parquet"]), - ("tablepath/part1=p1v2", 1, vec![]), - ("tablepath/part1=p1v2/part2=p2v1", 2, vec![]), - ("tablepath/part1=p1v2/part2=p2v2", 2, vec![]), - ("tablepath/part1=p1v3", 1, vec![]), - ("tablepath/part1=p1v3/part2=p2v1", 2, vec![]), - ] - ); - - let partitions = list_partitions( - store.as_ref(), - &ListingTableUrl::parse("file:///tablepath/").unwrap(), - 2, - None, - ) - .await - .expect("listing partitions failed"); - - assert_eq!( - &partitions - .iter() - .map(describe_partition) - .collect::>(), - &vec![ - ("tablepath", 0, vec![]), - ("tablepath/part1=p1v1", 1, vec!["file.parquet"]), - ("tablepath/part1=p1v2", 1, vec![]), - ("tablepath/part1=p1v3", 1, vec![]), - ( - "tablepath/part1=p1v2/part2=p2v1", - 2, - vec!["file1.parquet", "file2.parquet"] - ), - ("tablepath/part1=p1v2/part2=p2v2", 2, vec!["file4.parquet"]), - ("tablepath/part1=p1v3/part2=p2v1", 2, vec!["file3.parquet"]), - ] - ); -} diff --git a/datafusion/core/tests/catalog_listing/mod.rs b/datafusion/core/tests/catalog_listing/mod.rs deleted file mode 100644 index 9df4cc1fa3ac..000000000000 --- a/datafusion/core/tests/catalog_listing/mod.rs +++ /dev/null @@ -1,18 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -mod helpers; From 3dc6199a60757b1312b8e509b220a3d33e46aeca Mon Sep 17 00:00:00 2001 From: logan-keede Date: Thu, 6 Feb 2025 00:17:18 +0530 Subject: [PATCH 5/7] fix: public rexport --- datafusion/core/src/datasource/physical_plan/mod.rs | 5 +---- datafusion/core/tests/core_integration.rs | 2 -- 2 files changed, 1 insertion(+), 6 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/mod.rs b/datafusion/core/src/datasource/physical_plan/mod.rs index 6370457cc364..0a3d9b2cdbd9 100644 --- a/datafusion/core/src/datasource/physical_plan/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/mod.rs @@ -36,10 +36,7 @@ pub use self::parquet::{ParquetExec, ParquetFileMetrics, ParquetFileReaderFactor pub use arrow_file::ArrowExec; pub use avro::AvroExec; pub use csv::{CsvConfig, CsvExec, CsvExecBuilder, CsvOpener}; -pub use datafusion_catalog_listing::file_scan_config::{ - wrap_partition_type_in_dict, wrap_partition_value_in_dict, FileGroupDisplay, - FileScanConfig, -}; +pub use datafusion_catalog_listing::file_scan_config::*; use datafusion_expr::dml::InsertOp; pub use file_groups::FileGroupPartitioner; pub use file_stream::{FileOpenFuture, FileOpener, FileStream, OnError}; diff --git a/datafusion/core/tests/core_integration.rs b/datafusion/core/tests/core_integration.rs index fb182baf1f87..66b4103160e7 100644 --- a/datafusion/core/tests/core_integration.rs +++ b/datafusion/core/tests/core_integration.rs @@ -46,8 +46,6 @@ mod physical_optimizer; mod catalog; -mod catalog_listing; - #[cfg(test)] #[ctor::ctor] fn init() { From 403121e9dc16d749d03a281004457e067583b927 Mon Sep 17 00:00:00 2001 From: logan-keede Date: Thu, 6 Feb 2025 00:36:53 +0530 Subject: [PATCH 6/7] build test --- datafusion-examples/examples/custom_file_format.rs | 1 - datafusion/catalog-listing/src/file_scan_config.rs | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/datafusion-examples/examples/custom_file_format.rs b/datafusion-examples/examples/custom_file_format.rs index aa2df036235a..23dba8112ad2 100644 --- a/datafusion-examples/examples/custom_file_format.rs +++ b/datafusion-examples/examples/custom_file_format.rs @@ -38,7 +38,6 @@ use datafusion::{ MemTable, }, error::Result, - execution::context::SessionState, physical_plan::ExecutionPlan, prelude::SessionContext, }; diff --git a/datafusion/catalog-listing/src/file_scan_config.rs b/datafusion/catalog-listing/src/file_scan_config.rs index 00c0ce98a6fb..ad24bfc1b7ca 100644 --- a/datafusion/catalog-listing/src/file_scan_config.rs +++ b/datafusion/catalog-listing/src/file_scan_config.rs @@ -59,7 +59,7 @@ use log::{debug, warn}; /// {NUM_GROUPS groups: [[file1, file2,...], [fileN, fileM, ...], ...]} /// ``` #[derive(Debug)] -struct FileGroupsDisplay<'a>(&'a [Vec]); +pub struct FileGroupsDisplay<'a>(pub &'a [Vec]); impl DisplayAs for FileGroupsDisplay<'_> { fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> FmtResult { From 885c599f8198e08957b66c57573568312d7b36a8 Mon Sep 17 00:00:00 2001 From: logan-keede Date: Thu, 6 Feb 2025 02:09:51 +0530 Subject: [PATCH 7/7] fix: docs --- .../catalog-listing/src/file_scan_config.rs | 6 +- .../physical_plan/file_scan_config.rs | 1284 ----------------- .../core/src/datasource/physical_plan/mod.rs | 2 - .../datasource/physical_plan/statistics.rs | 287 ---- 4 files changed, 3 insertions(+), 1576 deletions(-) delete mode 100644 datafusion/core/src/datasource/physical_plan/file_scan_config.rs delete mode 100644 datafusion/core/src/datasource/physical_plan/statistics.rs diff --git a/datafusion/catalog-listing/src/file_scan_config.rs b/datafusion/catalog-listing/src/file_scan_config.rs index ad24bfc1b7ca..f42268502b82 100644 --- a/datafusion/catalog-listing/src/file_scan_config.rs +++ b/datafusion/catalog-listing/src/file_scan_config.rs @@ -298,7 +298,7 @@ fn get_projected_output_ordering( /// /// Use [`wrap_partition_value_in_dict`] to wrap a [`ScalarValue`] in the same say. /// -/// [`ListingTable`]: crate::datasource::listing::ListingTable +/// [`ListingTable`]: https://github.com/apache/datafusion/blob/main/datafusion/core/src/datasource/listing/table.rs#L707 pub fn wrap_partition_type_in_dict(val_type: DataType) -> DataType { DataType::Dictionary(Box::new(DataType::UInt16), Box::new(val_type)) } @@ -317,8 +317,8 @@ pub fn wrap_partition_value_in_dict(val: ScalarValue) -> ScalarValue { /// ``` /// # use std::sync::Arc; /// # use arrow_schema::Schema; -/// use datafusion::datasource::listing::PartitionedFile; -/// # use datafusion::datasource::physical_plan::FileScanConfig; +/// # use datafusion_catalog_listing::PartitionedFile; +/// # use datafusion_catalog_listing::file_scan_config::FileScanConfig; /// # use datafusion_execution::object_store::ObjectStoreUrl; /// # let file_schema = Arc::new(Schema::empty()); /// // create FileScan config for reading data from file:// diff --git a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs deleted file mode 100644 index 886492dbc231..000000000000 --- a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs +++ /dev/null @@ -1,1284 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -//! [`FileScanConfig`] to configure scanning of possibly partitioned -//! file sources. - -use std::{ - borrow::Cow, collections::HashMap, fmt::Debug, marker::PhantomData, mem::size_of, - sync::Arc, vec, -}; - -use super::{get_projected_output_ordering, statistics::MinMaxStatistics}; -use datafusion_catalog_listing::PartitionedFile; -use datafusion_execution::object_store::ObjectStoreUrl; -use datafusion_common::error::Result; -use datafusion_common::scalar::ScalarValue; - -use arrow::array::{ArrayData, BufferBuilder}; -use arrow::buffer::Buffer; -use arrow::datatypes::{ArrowNativeType, UInt16Type}; -use arrow_array::{ArrayRef, DictionaryArray, RecordBatch, RecordBatchOptions}; -use arrow_schema::{DataType, Field, Schema, SchemaRef}; -use datafusion_common::stats::Precision; -use datafusion_common::{ - exec_err, ColumnStatistics, Constraints, DataFusionError, Statistics, -}; -use datafusion_physical_expr::LexOrdering; - -use log::warn; - -/// Convert type to a type suitable for use as a [`ListingTable`] -/// partition column. Returns `Dictionary(UInt16, val_type)`, which is -/// a reasonable trade off between a reasonable number of partition -/// values and space efficiency. -/// -/// This use this to specify types for partition columns. However -/// you MAY also choose not to dictionary-encode the data or to use a -/// different dictionary type. -/// -/// Use [`wrap_partition_value_in_dict`] to wrap a [`ScalarValue`] in the same say. -/// -/// [`ListingTable`]: crate::datasource::listing::ListingTable -pub fn wrap_partition_type_in_dict(val_type: DataType) -> DataType { - DataType::Dictionary(Box::new(DataType::UInt16), Box::new(val_type)) -} - -/// Convert a [`ScalarValue`] of partition columns to a type, as -/// described in the documentation of [`wrap_partition_type_in_dict`], -/// which can wrap the types. -pub fn wrap_partition_value_in_dict(val: ScalarValue) -> ScalarValue { - ScalarValue::Dictionary(Box::new(DataType::UInt16), Box::new(val)) -} - -/// The base configurations to provide when creating a physical plan for -/// any given file format. -/// -/// # Example -/// ``` -/// # use std::sync::Arc; -/// # use arrow_schema::Schema; -/// use datafusion::datasource::listing::PartitionedFile; -/// # use datafusion::datasource::physical_plan::FileScanConfig; -/// # use datafusion_execution::object_store::ObjectStoreUrl; -/// # let file_schema = Arc::new(Schema::empty()); -/// // create FileScan config for reading data from file:// -/// let object_store_url = ObjectStoreUrl::local_filesystem(); -/// let config = FileScanConfig::new(object_store_url, file_schema) -/// .with_limit(Some(1000)) // read only the first 1000 records -/// .with_projection(Some(vec![2, 3])) // project columns 2 and 3 -/// // Read /tmp/file1.parquet with known size of 1234 bytes in a single group -/// .with_file(PartitionedFile::new("file1.parquet", 1234)) -/// // Read /tmp/file2.parquet 56 bytes and /tmp/file3.parquet 78 bytes -/// // in a single row group -/// .with_file_group(vec![ -/// PartitionedFile::new("file2.parquet", 56), -/// PartitionedFile::new("file3.parquet", 78), -/// ]); -/// ``` -#[derive(Clone)] -pub struct FileScanConfig { - /// Object store URL, used to get an [`ObjectStore`] instance from - /// [`RuntimeEnv::object_store`] - /// - /// This `ObjectStoreUrl` should be the prefix of the absolute url for files - /// as `file://` or `s3://my_bucket`. It should not include the path to the - /// file itself. The relevant URL prefix must be registered via - /// [`RuntimeEnv::register_object_store`] - /// - /// [`ObjectStore`]: object_store::ObjectStore - /// [`RuntimeEnv::register_object_store`]: datafusion_execution::runtime_env::RuntimeEnv::register_object_store - /// [`RuntimeEnv::object_store`]: datafusion_execution::runtime_env::RuntimeEnv::object_store - pub object_store_url: ObjectStoreUrl, - /// Schema before `projection` is applied. It contains the all columns that may - /// appear in the files. It does not include table partition columns - /// that may be added. - pub file_schema: SchemaRef, - /// List of files to be processed, grouped into partitions - /// - /// Each file must have a schema of `file_schema` or a subset. If - /// a particular file has a subset, the missing columns are - /// padded with NULLs. - /// - /// DataFusion may attempt to read each partition of files - /// concurrently, however files *within* a partition will be read - /// sequentially, one after the next. - pub file_groups: Vec>, - /// Table constraints - pub constraints: Constraints, - /// Estimated overall statistics of the files, taking `filters` into account. - /// Defaults to [`Statistics::new_unknown`]. - pub statistics: Statistics, - /// Columns on which to project the data. Indexes that are higher than the - /// number of columns of `file_schema` refer to `table_partition_cols`. - pub projection: Option>, - /// The maximum number of records to read from this plan. If `None`, - /// all records after filtering are returned. - pub limit: Option, - /// The partitioning columns - pub table_partition_cols: Vec, - /// All equivalent lexicographical orderings that describe the schema. - pub output_ordering: Vec, -} - -impl FileScanConfig { - /// Create a new `FileScanConfig` with default settings for scanning files. - /// - /// See example on [`FileScanConfig`] - /// - /// No file groups are added by default. See [`Self::with_file`], [`Self::with_file_group]` and - /// [`Self::with_file_groups`]. - /// - /// # Parameters: - /// * `object_store_url`: See [`Self::object_store_url`] - /// * `file_schema`: See [`Self::file_schema`] - pub fn new(object_store_url: ObjectStoreUrl, file_schema: SchemaRef) -> Self { - let statistics = Statistics::new_unknown(&file_schema); - Self { - object_store_url, - file_schema, - file_groups: vec![], - constraints: Constraints::empty(), - statistics, - projection: None, - limit: None, - table_partition_cols: vec![], - output_ordering: vec![], - } - } - - /// Set the table constraints of the files - pub fn with_constraints(mut self, constraints: Constraints) -> Self { - self.constraints = constraints; - self - } - - /// Set the statistics of the files - pub fn with_statistics(mut self, statistics: Statistics) -> Self { - self.statistics = statistics; - self - } - - /// Set the projection of the files - pub fn with_projection(mut self, projection: Option>) -> Self { - self.projection = projection; - self - } - - /// Set the limit of the files - pub fn with_limit(mut self, limit: Option) -> Self { - self.limit = limit; - self - } - - /// Add a file as a single group - /// - /// See [Self::file_groups] for more information. - pub fn with_file(self, file: PartitionedFile) -> Self { - self.with_file_group(vec![file]) - } - - /// Add the file groups - /// - /// See [Self::file_groups] for more information. - pub fn with_file_groups( - mut self, - mut file_groups: Vec>, - ) -> Self { - self.file_groups.append(&mut file_groups); - self - } - - /// Add a new file group - /// - /// See [Self::file_groups] for more information - pub fn with_file_group(mut self, file_group: Vec) -> Self { - self.file_groups.push(file_group); - self - } - - /// Set the partitioning columns of the files - pub fn with_table_partition_cols(mut self, table_partition_cols: Vec) -> Self { - self.table_partition_cols = table_partition_cols; - self - } - - /// Set the output ordering of the files - pub fn with_output_ordering(mut self, output_ordering: Vec) -> Self { - self.output_ordering = output_ordering; - self - } - - /// Project the schema, constraints, and the statistics on the given column indices - pub fn project(&self) -> (SchemaRef, Constraints, Statistics, Vec) { - if self.projection.is_none() && self.table_partition_cols.is_empty() { - return ( - Arc::clone(&self.file_schema), - self.constraints.clone(), - self.statistics.clone(), - self.output_ordering.clone(), - ); - } - - let proj_indices = if let Some(proj) = &self.projection { - proj - } else { - let len = self.file_schema.fields().len() + self.table_partition_cols.len(); - &(0..len).collect::>() - }; - - let mut table_fields = vec![]; - let mut table_cols_stats = vec![]; - for idx in proj_indices { - if *idx < self.file_schema.fields().len() { - let field = self.file_schema.field(*idx); - table_fields.push(field.clone()); - table_cols_stats.push(self.statistics.column_statistics[*idx].clone()) - } else { - let partition_idx = idx - self.file_schema.fields().len(); - table_fields.push(self.table_partition_cols[partition_idx].to_owned()); - // TODO provide accurate stat for partition column (#1186) - table_cols_stats.push(ColumnStatistics::new_unknown()) - } - } - - let table_stats = Statistics { - num_rows: self.statistics.num_rows, - // TODO correct byte size? - total_byte_size: Precision::Absent, - column_statistics: table_cols_stats, - }; - - let projected_schema = Arc::new(Schema::new_with_metadata( - table_fields, - self.file_schema.metadata().clone(), - )); - - let projected_constraints = self - .constraints - .project(proj_indices) - .unwrap_or_else(Constraints::empty); - - let projected_output_ordering = - get_projected_output_ordering(self, &projected_schema); - - ( - projected_schema, - projected_constraints, - table_stats, - projected_output_ordering, - ) - } - - #[cfg_attr(not(feature = "avro"), allow(unused))] // Only used by avro - pub(crate) fn projected_file_column_names(&self) -> Option> { - self.projection.as_ref().map(|p| { - p.iter() - .filter(|col_idx| **col_idx < self.file_schema.fields().len()) - .map(|col_idx| self.file_schema.field(*col_idx).name()) - .cloned() - .collect() - }) - } - - /// Projects only file schema, ignoring partition columns - pub(crate) fn projected_file_schema(&self) -> SchemaRef { - let fields = self.file_column_projection_indices().map(|indices| { - indices - .iter() - .map(|col_idx| self.file_schema.field(*col_idx)) - .cloned() - .collect::>() - }); - - fields.map_or_else( - || Arc::clone(&self.file_schema), - |f| { - Arc::new(Schema::new_with_metadata( - f, - self.file_schema.metadata.clone(), - )) - }, - ) - } - - pub(crate) fn file_column_projection_indices(&self) -> Option> { - self.projection.as_ref().map(|p| { - p.iter() - .filter(|col_idx| **col_idx < self.file_schema.fields().len()) - .copied() - .collect() - }) - } - - /// Attempts to do a bin-packing on files into file groups, such that any two files - /// in a file group are ordered and non-overlapping with respect to their statistics. - /// It will produce the smallest number of file groups possible. - pub fn split_groups_by_statistics( - table_schema: &SchemaRef, - file_groups: &[Vec], - sort_order: &LexOrdering, - ) -> Result>> { - let flattened_files = file_groups.iter().flatten().collect::>(); - // First Fit: - // * Choose the first file group that a file can be placed into. - // * If it fits into no existing file groups, create a new one. - // - // By sorting files by min values and then applying first-fit bin packing, - // we can produce the smallest number of file groups such that - // files within a group are in order and non-overlapping. - // - // Source: Applied Combinatorics (Keller and Trotter), Chapter 6.8 - // https://www.appliedcombinatorics.org/book/s_posets_dilworth-intord.html - - if flattened_files.is_empty() { - return Ok(vec![]); - } - - let statistics = MinMaxStatistics::new_from_files( - sort_order, - table_schema, - None, - flattened_files.iter().copied(), - ) - .map_err(|e| { - e.context("construct min/max statistics for split_groups_by_statistics") - })?; - - let indices_sorted_by_min = statistics.min_values_sorted(); - let mut file_groups_indices: Vec> = vec![]; - - for (idx, min) in indices_sorted_by_min { - let file_group_to_insert = file_groups_indices.iter_mut().find(|group| { - // If our file is non-overlapping and comes _after_ the last file, - // it fits in this file group. - min > statistics.max( - *group - .last() - .expect("groups should be nonempty at construction"), - ) - }); - match file_group_to_insert { - Some(group) => group.push(idx), - None => file_groups_indices.push(vec![idx]), - } - } - - // Assemble indices back into groups of PartitionedFiles - Ok(file_groups_indices - .into_iter() - .map(|file_group_indices| { - file_group_indices - .into_iter() - .map(|idx| flattened_files[idx].clone()) - .collect() - }) - .collect()) - } -} - -/// A helper that projects partition columns into the file record batches. -/// -/// One interesting trick is the usage of a cache for the key buffers of the partition column -/// dictionaries. Indeed, the partition columns are constant, so the dictionaries that represent them -/// have all their keys equal to 0. This enables us to re-use the same "all-zero" buffer across batches, -/// which makes the space consumption of the partition columns O(batch_size) instead of O(record_count). -pub struct PartitionColumnProjector { - /// An Arrow buffer initialized to zeros that represents the key array of all partition - /// columns (partition columns are materialized by dictionary arrays with only one - /// value in the dictionary, thus all the keys are equal to zero). - key_buffer_cache: ZeroBufferGenerators, - /// Mapping between the indexes in the list of partition columns and the target - /// schema. Sorted by index in the target schema so that we can iterate on it to - /// insert the partition columns in the target record batch. - projected_partition_indexes: Vec<(usize, usize)>, - /// The schema of the table once the projection was applied. - projected_schema: SchemaRef, -} - -impl PartitionColumnProjector { - // Create a projector to insert the partitioning columns into batches read from files - // - `projected_schema`: the target schema with both file and partitioning columns - // - `table_partition_cols`: all the partitioning column names - pub fn new(projected_schema: SchemaRef, table_partition_cols: &[String]) -> Self { - let mut idx_map = HashMap::new(); - for (partition_idx, partition_name) in table_partition_cols.iter().enumerate() { - if let Ok(schema_idx) = projected_schema.index_of(partition_name) { - idx_map.insert(partition_idx, schema_idx); - } - } - - let mut projected_partition_indexes: Vec<_> = idx_map.into_iter().collect(); - projected_partition_indexes.sort_by(|(_, a), (_, b)| a.cmp(b)); - - Self { - projected_partition_indexes, - key_buffer_cache: Default::default(), - projected_schema, - } - } - - // Transform the batch read from the file by inserting the partitioning columns - // to the right positions as deduced from `projected_schema` - // - `file_batch`: batch read from the file, with internal projection applied - // - `partition_values`: the list of partition values, one for each partition column - pub fn project( - &mut self, - file_batch: RecordBatch, - partition_values: &[ScalarValue], - ) -> Result { - let expected_cols = - self.projected_schema.fields().len() - self.projected_partition_indexes.len(); - - if file_batch.columns().len() != expected_cols { - return exec_err!( - "Unexpected batch schema from file, expected {} cols but got {}", - expected_cols, - file_batch.columns().len() - ); - } - - let mut cols = file_batch.columns().to_vec(); - for &(pidx, sidx) in &self.projected_partition_indexes { - let p_value = - partition_values - .get(pidx) - .ok_or(DataFusionError::Execution( - "Invalid partitioning found on disk".to_string(), - ))?; - - let mut partition_value = Cow::Borrowed(p_value); - - // check if user forgot to dict-encode the partition value - let field = self.projected_schema.field(sidx); - let expected_data_type = field.data_type(); - let actual_data_type = partition_value.data_type(); - if let DataType::Dictionary(key_type, _) = expected_data_type { - if !matches!(actual_data_type, DataType::Dictionary(_, _)) { - warn!("Partition value for column {} was not dictionary-encoded, applied auto-fix.", field.name()); - partition_value = Cow::Owned(ScalarValue::Dictionary( - key_type.clone(), - Box::new(partition_value.as_ref().clone()), - )); - } - } - - cols.insert( - sidx, - create_output_array( - &mut self.key_buffer_cache, - partition_value.as_ref(), - file_batch.num_rows(), - )?, - ) - } - - RecordBatch::try_new_with_options( - Arc::clone(&self.projected_schema), - cols, - &RecordBatchOptions::new().with_row_count(Some(file_batch.num_rows())), - ) - .map_err(Into::into) - } -} - -#[derive(Debug, Default)] -struct ZeroBufferGenerators { - gen_i8: ZeroBufferGenerator, - gen_i16: ZeroBufferGenerator, - gen_i32: ZeroBufferGenerator, - gen_i64: ZeroBufferGenerator, - gen_u8: ZeroBufferGenerator, - gen_u16: ZeroBufferGenerator, - gen_u32: ZeroBufferGenerator, - gen_u64: ZeroBufferGenerator, -} - -/// Generate a arrow [`Buffer`] that contains zero values. -#[derive(Debug, Default)] -struct ZeroBufferGenerator -where - T: ArrowNativeType, -{ - cache: Option, - _t: PhantomData, -} - -impl ZeroBufferGenerator -where - T: ArrowNativeType, -{ - const SIZE: usize = size_of::(); - - fn get_buffer(&mut self, n_vals: usize) -> Buffer { - match &mut self.cache { - Some(buf) if buf.len() >= n_vals * Self::SIZE => { - buf.slice_with_length(0, n_vals * Self::SIZE) - } - _ => { - let mut key_buffer_builder = BufferBuilder::::new(n_vals); - key_buffer_builder.advance(n_vals); // keys are all 0 - self.cache.insert(key_buffer_builder.finish()).clone() - } - } - } -} - -fn create_dict_array( - buffer_gen: &mut ZeroBufferGenerator, - dict_val: &ScalarValue, - len: usize, - data_type: DataType, -) -> Result -where - T: ArrowNativeType, -{ - let dict_vals = dict_val.to_array()?; - - let sliced_key_buffer = buffer_gen.get_buffer(len); - - // assemble pieces together - let mut builder = ArrayData::builder(data_type) - .len(len) - .add_buffer(sliced_key_buffer); - builder = builder.add_child_data(dict_vals.to_data()); - Ok(Arc::new(DictionaryArray::::from( - builder.build().unwrap(), - ))) -} - -fn create_output_array( - key_buffer_cache: &mut ZeroBufferGenerators, - val: &ScalarValue, - len: usize, -) -> Result { - if let ScalarValue::Dictionary(key_type, dict_val) = &val { - match key_type.as_ref() { - DataType::Int8 => { - return create_dict_array( - &mut key_buffer_cache.gen_i8, - dict_val, - len, - val.data_type(), - ); - } - DataType::Int16 => { - return create_dict_array( - &mut key_buffer_cache.gen_i16, - dict_val, - len, - val.data_type(), - ); - } - DataType::Int32 => { - return create_dict_array( - &mut key_buffer_cache.gen_i32, - dict_val, - len, - val.data_type(), - ); - } - DataType::Int64 => { - return create_dict_array( - &mut key_buffer_cache.gen_i64, - dict_val, - len, - val.data_type(), - ); - } - DataType::UInt8 => { - return create_dict_array( - &mut key_buffer_cache.gen_u8, - dict_val, - len, - val.data_type(), - ); - } - DataType::UInt16 => { - return create_dict_array( - &mut key_buffer_cache.gen_u16, - dict_val, - len, - val.data_type(), - ); - } - DataType::UInt32 => { - return create_dict_array( - &mut key_buffer_cache.gen_u32, - dict_val, - len, - val.data_type(), - ); - } - DataType::UInt64 => { - return create_dict_array( - &mut key_buffer_cache.gen_u64, - dict_val, - len, - val.data_type(), - ); - } - _ => {} - } - } - - val.to_array_of_size(len) -} - -#[cfg(test)] -mod tests { - use arrow_array::Int32Array; - - use super::*; - use crate::{test::columns, test_util::aggr_test_schema}; - - #[test] - fn physical_plan_config_no_projection() { - let file_schema = aggr_test_schema(); - let conf = config_for_projection( - Arc::clone(&file_schema), - None, - Statistics::new_unknown(&file_schema), - to_partition_cols(vec![( - "date".to_owned(), - wrap_partition_type_in_dict(DataType::Utf8), - )]), - ); - - let (proj_schema, _, proj_statistics, _) = conf.project(); - assert_eq!(proj_schema.fields().len(), file_schema.fields().len() + 1); - assert_eq!( - proj_schema.field(file_schema.fields().len()).name(), - "date", - "partition columns are the last columns" - ); - assert_eq!( - proj_statistics.column_statistics.len(), - file_schema.fields().len() + 1 - ); - // TODO implement tests for partition column statistics once implemented - - let col_names = conf.projected_file_column_names(); - assert_eq!(col_names, None); - - let col_indices = conf.file_column_projection_indices(); - assert_eq!(col_indices, None); - } - - #[test] - fn physical_plan_config_no_projection_tab_cols_as_field() { - let file_schema = aggr_test_schema(); - - // make a table_partition_col as a field - let table_partition_col = - Field::new("date", wrap_partition_type_in_dict(DataType::Utf8), true) - .with_metadata(HashMap::from_iter(vec![( - "key_whatever".to_owned(), - "value_whatever".to_owned(), - )])); - - let conf = config_for_projection( - Arc::clone(&file_schema), - None, - Statistics::new_unknown(&file_schema), - vec![table_partition_col.clone()], - ); - - // verify the proj_schema includes the last column and exactly the same the field it is defined - let (proj_schema, _, _, _) = conf.project(); - assert_eq!(proj_schema.fields().len(), file_schema.fields().len() + 1); - assert_eq!( - *proj_schema.field(file_schema.fields().len()), - table_partition_col, - "partition columns are the last columns and ust have all values defined in created field" - ); - } - - #[test] - fn physical_plan_config_with_projection() { - let file_schema = aggr_test_schema(); - let conf = config_for_projection( - Arc::clone(&file_schema), - Some(vec![file_schema.fields().len(), 0]), - Statistics { - num_rows: Precision::Inexact(10), - // assign the column index to distinct_count to help assert - // the source statistic after the projection - column_statistics: (0..file_schema.fields().len()) - .map(|i| ColumnStatistics { - distinct_count: Precision::Inexact(i), - ..Default::default() - }) - .collect(), - total_byte_size: Precision::Absent, - }, - to_partition_cols(vec![( - "date".to_owned(), - wrap_partition_type_in_dict(DataType::Utf8), - )]), - ); - - let (proj_schema, _, proj_statistics, _) = conf.project(); - assert_eq!( - columns(&proj_schema), - vec!["date".to_owned(), "c1".to_owned()] - ); - let proj_stat_cols = proj_statistics.column_statistics; - assert_eq!(proj_stat_cols.len(), 2); - // TODO implement tests for proj_stat_cols[0] once partition column - // statistics are implemented - assert_eq!(proj_stat_cols[1].distinct_count, Precision::Inexact(0)); - - let col_names = conf.projected_file_column_names(); - assert_eq!(col_names, Some(vec!["c1".to_owned()])); - - let col_indices = conf.file_column_projection_indices(); - assert_eq!(col_indices, Some(vec![0])); - } - - #[test] - fn partition_column_projector() { - let file_batch = build_table_i32( - ("a", &vec![0, 1, 2]), - ("b", &vec![-2, -1, 0]), - ("c", &vec![10, 11, 12]), - ); - let partition_cols = vec![ - ( - "year".to_owned(), - wrap_partition_type_in_dict(DataType::Utf8), - ), - ( - "month".to_owned(), - wrap_partition_type_in_dict(DataType::Utf8), - ), - ( - "day".to_owned(), - wrap_partition_type_in_dict(DataType::Utf8), - ), - ]; - // create a projected schema - let conf = config_for_projection( - file_batch.schema(), - // keep all cols from file and 2 from partitioning - Some(vec![ - 0, - 1, - 2, - file_batch.schema().fields().len(), - file_batch.schema().fields().len() + 2, - ]), - Statistics::new_unknown(&file_batch.schema()), - to_partition_cols(partition_cols.clone()), - ); - let (proj_schema, ..) = conf.project(); - // created a projector for that projected schema - let mut proj = PartitionColumnProjector::new( - proj_schema, - &partition_cols - .iter() - .map(|x| x.0.clone()) - .collect::>(), - ); - - // project first batch - let projected_batch = proj - .project( - // file_batch is ok here because we kept all the file cols in the projection - file_batch, - &[ - wrap_partition_value_in_dict(ScalarValue::from("2021")), - wrap_partition_value_in_dict(ScalarValue::from("10")), - wrap_partition_value_in_dict(ScalarValue::from("26")), - ], - ) - .expect("Projection of partition columns into record batch failed"); - let expected = [ - "+---+----+----+------+-----+", - "| a | b | c | year | day |", - "+---+----+----+------+-----+", - "| 0 | -2 | 10 | 2021 | 26 |", - "| 1 | -1 | 11 | 2021 | 26 |", - "| 2 | 0 | 12 | 2021 | 26 |", - "+---+----+----+------+-----+", - ]; - crate::assert_batches_eq!(expected, &[projected_batch]); - - // project another batch that is larger than the previous one - let file_batch = build_table_i32( - ("a", &vec![5, 6, 7, 8, 9]), - ("b", &vec![-10, -9, -8, -7, -6]), - ("c", &vec![12, 13, 14, 15, 16]), - ); - let projected_batch = proj - .project( - // file_batch is ok here because we kept all the file cols in the projection - file_batch, - &[ - wrap_partition_value_in_dict(ScalarValue::from("2021")), - wrap_partition_value_in_dict(ScalarValue::from("10")), - wrap_partition_value_in_dict(ScalarValue::from("27")), - ], - ) - .expect("Projection of partition columns into record batch failed"); - let expected = [ - "+---+-----+----+------+-----+", - "| a | b | c | year | day |", - "+---+-----+----+------+-----+", - "| 5 | -10 | 12 | 2021 | 27 |", - "| 6 | -9 | 13 | 2021 | 27 |", - "| 7 | -8 | 14 | 2021 | 27 |", - "| 8 | -7 | 15 | 2021 | 27 |", - "| 9 | -6 | 16 | 2021 | 27 |", - "+---+-----+----+------+-----+", - ]; - crate::assert_batches_eq!(expected, &[projected_batch]); - - // project another batch that is smaller than the previous one - let file_batch = build_table_i32( - ("a", &vec![0, 1, 3]), - ("b", &vec![2, 3, 4]), - ("c", &vec![4, 5, 6]), - ); - let projected_batch = proj - .project( - // file_batch is ok here because we kept all the file cols in the projection - file_batch, - &[ - wrap_partition_value_in_dict(ScalarValue::from("2021")), - wrap_partition_value_in_dict(ScalarValue::from("10")), - wrap_partition_value_in_dict(ScalarValue::from("28")), - ], - ) - .expect("Projection of partition columns into record batch failed"); - let expected = [ - "+---+---+---+------+-----+", - "| a | b | c | year | day |", - "+---+---+---+------+-----+", - "| 0 | 2 | 4 | 2021 | 28 |", - "| 1 | 3 | 5 | 2021 | 28 |", - "| 3 | 4 | 6 | 2021 | 28 |", - "+---+---+---+------+-----+", - ]; - crate::assert_batches_eq!(expected, &[projected_batch]); - - // forgot to dictionary-wrap the scalar value - let file_batch = build_table_i32( - ("a", &vec![0, 1, 2]), - ("b", &vec![-2, -1, 0]), - ("c", &vec![10, 11, 12]), - ); - let projected_batch = proj - .project( - // file_batch is ok here because we kept all the file cols in the projection - file_batch, - &[ - ScalarValue::from("2021"), - ScalarValue::from("10"), - ScalarValue::from("26"), - ], - ) - .expect("Projection of partition columns into record batch failed"); - let expected = [ - "+---+----+----+------+-----+", - "| a | b | c | year | day |", - "+---+----+----+------+-----+", - "| 0 | -2 | 10 | 2021 | 26 |", - "| 1 | -1 | 11 | 2021 | 26 |", - "| 2 | 0 | 12 | 2021 | 26 |", - "+---+----+----+------+-----+", - ]; - crate::assert_batches_eq!(expected, &[projected_batch]); - } - - #[test] - fn test_projected_file_schema_with_partition_col() { - let schema = aggr_test_schema(); - let partition_cols = vec![ - ( - "part1".to_owned(), - wrap_partition_type_in_dict(DataType::Utf8), - ), - ( - "part2".to_owned(), - wrap_partition_type_in_dict(DataType::Utf8), - ), - ]; - - // Projected file schema for config with projection including partition column - let projection = config_for_projection( - schema.clone(), - Some(vec![0, 3, 5, schema.fields().len()]), - Statistics::new_unknown(&schema), - to_partition_cols(partition_cols), - ) - .projected_file_schema(); - - // Assert partition column filtered out in projected file schema - let expected_columns = vec!["c1", "c4", "c6"]; - let actual_columns = projection - .fields() - .iter() - .map(|f| f.name().clone()) - .collect::>(); - assert_eq!(expected_columns, actual_columns); - } - - #[test] - fn test_projected_file_schema_without_projection() { - let schema = aggr_test_schema(); - let partition_cols = vec![ - ( - "part1".to_owned(), - wrap_partition_type_in_dict(DataType::Utf8), - ), - ( - "part2".to_owned(), - wrap_partition_type_in_dict(DataType::Utf8), - ), - ]; - - // Projected file schema for config without projection - let projection = config_for_projection( - schema.clone(), - None, - Statistics::new_unknown(&schema), - to_partition_cols(partition_cols), - ) - .projected_file_schema(); - - // Assert projected file schema is equal to file schema - assert_eq!(projection.fields(), schema.fields()); - } - - #[test] - fn test_split_groups_by_statistics() -> Result<()> { - use chrono::TimeZone; - use datafusion_common::DFSchema; - use datafusion_expr::execution_props::ExecutionProps; - use object_store::{path::Path, ObjectMeta}; - - struct File { - name: &'static str, - date: &'static str, - statistics: Vec>, - } - impl File { - fn new( - name: &'static str, - date: &'static str, - statistics: Vec>, - ) -> Self { - Self { - name, - date, - statistics, - } - } - } - - struct TestCase { - name: &'static str, - file_schema: Schema, - files: Vec, - sort: Vec, - expected_result: Result>, &'static str>, - } - - use datafusion_expr::col; - let cases = vec![ - TestCase { - name: "test sort", - file_schema: Schema::new(vec![Field::new( - "value".to_string(), - DataType::Float64, - false, - )]), - files: vec![ - File::new("0", "2023-01-01", vec![Some((0.00, 0.49))]), - File::new("1", "2023-01-01", vec![Some((0.50, 1.00))]), - File::new("2", "2023-01-02", vec![Some((0.00, 1.00))]), - ], - sort: vec![col("value").sort(true, false)], - expected_result: Ok(vec![vec!["0", "1"], vec!["2"]]), - }, - // same input but file '2' is in the middle - // test that we still order correctly - TestCase { - name: "test sort with files ordered differently", - file_schema: Schema::new(vec![Field::new( - "value".to_string(), - DataType::Float64, - false, - )]), - files: vec![ - File::new("0", "2023-01-01", vec![Some((0.00, 0.49))]), - File::new("2", "2023-01-02", vec![Some((0.00, 1.00))]), - File::new("1", "2023-01-01", vec![Some((0.50, 1.00))]), - ], - sort: vec![col("value").sort(true, false)], - expected_result: Ok(vec![vec!["0", "1"], vec!["2"]]), - }, - TestCase { - name: "reverse sort", - file_schema: Schema::new(vec![Field::new( - "value".to_string(), - DataType::Float64, - false, - )]), - files: vec![ - File::new("0", "2023-01-01", vec![Some((0.00, 0.49))]), - File::new("1", "2023-01-01", vec![Some((0.50, 1.00))]), - File::new("2", "2023-01-02", vec![Some((0.00, 1.00))]), - ], - sort: vec![col("value").sort(false, true)], - expected_result: Ok(vec![vec!["1", "0"], vec!["2"]]), - }, - // reject nullable sort columns - TestCase { - name: "no nullable sort columns", - file_schema: Schema::new(vec![Field::new( - "value".to_string(), - DataType::Float64, - true, // should fail because nullable - )]), - files: vec![ - File::new("0", "2023-01-01", vec![Some((0.00, 0.49))]), - File::new("1", "2023-01-01", vec![Some((0.50, 1.00))]), - File::new("2", "2023-01-02", vec![Some((0.00, 1.00))]), - ], - sort: vec![col("value").sort(true, false)], - expected_result: Err("construct min/max statistics for split_groups_by_statistics\ncaused by\nbuild min rows\ncaused by\ncreate sorting columns\ncaused by\nError during planning: cannot sort by nullable column") - }, - TestCase { - name: "all three non-overlapping", - file_schema: Schema::new(vec![Field::new( - "value".to_string(), - DataType::Float64, - false, - )]), - files: vec![ - File::new("0", "2023-01-01", vec![Some((0.00, 0.49))]), - File::new("1", "2023-01-01", vec![Some((0.50, 0.99))]), - File::new("2", "2023-01-02", vec![Some((1.00, 1.49))]), - ], - sort: vec![col("value").sort(true, false)], - expected_result: Ok(vec![vec!["0", "1", "2"]]), - }, - TestCase { - name: "all three overlapping", - file_schema: Schema::new(vec![Field::new( - "value".to_string(), - DataType::Float64, - false, - )]), - files: vec![ - File::new("0", "2023-01-01", vec![Some((0.00, 0.49))]), - File::new("1", "2023-01-01", vec![Some((0.00, 0.49))]), - File::new("2", "2023-01-02", vec![Some((0.00, 0.49))]), - ], - sort: vec![col("value").sort(true, false)], - expected_result: Ok(vec![vec!["0"], vec!["1"], vec!["2"]]), - }, - TestCase { - name: "empty input", - file_schema: Schema::new(vec![Field::new( - "value".to_string(), - DataType::Float64, - false, - )]), - files: vec![], - sort: vec![col("value").sort(true, false)], - expected_result: Ok(vec![]), - }, - TestCase { - name: "one file missing statistics", - file_schema: Schema::new(vec![Field::new( - "value".to_string(), - DataType::Float64, - false, - )]), - files: vec![ - File::new("0", "2023-01-01", vec![Some((0.00, 0.49))]), - File::new("1", "2023-01-01", vec![Some((0.00, 0.49))]), - File::new("2", "2023-01-02", vec![None]), - ], - sort: vec![col("value").sort(true, false)], - expected_result: Err("construct min/max statistics for split_groups_by_statistics\ncaused by\ncollect min/max values\ncaused by\nget min/max for column: 'value'\ncaused by\nError during planning: statistics not found"), - }, - ]; - - for case in cases { - let table_schema = Arc::new(Schema::new( - case.file_schema - .fields() - .clone() - .into_iter() - .cloned() - .chain(Some(Arc::new(Field::new( - "date".to_string(), - DataType::Utf8, - false, - )))) - .collect::>(), - )); - let sort_order = LexOrdering::from( - case.sort - .into_iter() - .map(|expr| { - crate::physical_planner::create_physical_sort_expr( - &expr, - &DFSchema::try_from(table_schema.as_ref().clone())?, - &ExecutionProps::default(), - ) - }) - .collect::>>()?, - ); - - let partitioned_files = - case.files.into_iter().map(From::from).collect::>(); - let result = FileScanConfig::split_groups_by_statistics( - &table_schema, - &[partitioned_files.clone()], - &sort_order, - ); - let results_by_name = result - .as_ref() - .map(|file_groups| { - file_groups - .iter() - .map(|file_group| { - file_group - .iter() - .map(|file| { - partitioned_files - .iter() - .find_map(|f| { - if f.object_meta == file.object_meta { - Some( - f.object_meta - .location - .as_ref() - .rsplit('/') - .next() - .unwrap() - .trim_end_matches(".parquet"), - ) - } else { - None - } - }) - .unwrap() - }) - .collect::>() - }) - .collect::>() - }) - .map_err(|e| e.strip_backtrace().leak() as &'static str); - - assert_eq!(results_by_name, case.expected_result, "{}", case.name); - } - - return Ok(()); - - impl From for PartitionedFile { - fn from(file: File) -> Self { - PartitionedFile { - object_meta: ObjectMeta { - location: Path::from(format!( - "data/date={}/{}.parquet", - file.date, file.name - )), - last_modified: chrono::Utc.timestamp_nanos(0), - size: 0, - e_tag: None, - version: None, - }, - partition_values: vec![ScalarValue::from(file.date)], - range: None, - statistics: Some(Statistics { - num_rows: Precision::Absent, - total_byte_size: Precision::Absent, - column_statistics: file - .statistics - .into_iter() - .map(|stats| { - stats - .map(|(min, max)| ColumnStatistics { - min_value: Precision::Exact(ScalarValue::from( - min, - )), - max_value: Precision::Exact(ScalarValue::from( - max, - )), - ..Default::default() - }) - .unwrap_or_default() - }) - .collect::>(), - }), - extensions: None, - metadata_size_hint: None, - } - } - } - } - - // sets default for configs that play no role in projections - fn config_for_projection( - file_schema: SchemaRef, - projection: Option>, - statistics: Statistics, - table_partition_cols: Vec, - ) -> FileScanConfig { - FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), file_schema) - .with_projection(projection) - .with_statistics(statistics) - .with_table_partition_cols(table_partition_cols) - } - - /// Convert partition columns from Vec to Vec - fn to_partition_cols(table_partition_cols: Vec<(String, DataType)>) -> Vec { - table_partition_cols - .iter() - .map(|(name, dtype)| Field::new(name, dtype.clone(), false)) - .collect::>() - } - - /// returns record batch with 3 columns of i32 in memory - pub fn build_table_i32( - a: (&str, &Vec), - b: (&str, &Vec), - c: (&str, &Vec), - ) -> RecordBatch { - let schema = Schema::new(vec![ - Field::new(a.0, DataType::Int32, false), - Field::new(b.0, DataType::Int32, false), - Field::new(c.0, DataType::Int32, false), - ]); - - RecordBatch::try_new( - Arc::new(schema), - vec![ - Arc::new(Int32Array::from(a.1.clone())), - Arc::new(Int32Array::from(b.1.clone())), - Arc::new(Int32Array::from(c.1.clone())), - ], - ) - .unwrap() - } -} diff --git a/datafusion/core/src/datasource/physical_plan/mod.rs b/datafusion/core/src/datasource/physical_plan/mod.rs index 0a3d9b2cdbd9..90e0b5bc6e4a 100644 --- a/datafusion/core/src/datasource/physical_plan/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/mod.rs @@ -21,12 +21,10 @@ mod arrow_file; mod avro; mod csv; mod file_groups; -// mod file_scan_config; mod file_stream; mod json; #[cfg(feature = "parquet")] pub mod parquet; -// mod statistics; pub(crate) use self::csv::plan_to_csv; pub(crate) use self::json::plan_to_json; diff --git a/datafusion/core/src/datasource/physical_plan/statistics.rs b/datafusion/core/src/datasource/physical_plan/statistics.rs deleted file mode 100644 index b90c63537b3a..000000000000 --- a/datafusion/core/src/datasource/physical_plan/statistics.rs +++ /dev/null @@ -1,287 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -/*! - * - * Use statistics to optimize physical planning. - * - * Currently, this module houses code to sort file groups if they are non-overlapping with - * respect to the required sort order. See [`MinMaxStatistics`] - * -*/ - -use std::sync::Arc; - -use datafusion_catalog_listing::PartitionedFile; - -use arrow::{ - compute::SortColumn, - row::{Row, Rows}, -}; -use arrow_array::RecordBatch; -use arrow_schema::SchemaRef; -use datafusion_common::{plan_err, DataFusionError, Result}; -use datafusion_physical_expr::{expressions::Column, PhysicalSortExpr}; -use datafusion_physical_expr_common::sort_expr::LexOrdering; - -/// A normalized representation of file min/max statistics that allows for efficient sorting & comparison. -/// The min/max values are ordered by [`Self::sort_order`]. -/// Furthermore, any columns that are reversed in the sort order have their min/max values swapped. -pub(crate) struct MinMaxStatistics { - min_by_sort_order: Rows, - max_by_sort_order: Rows, - sort_order: LexOrdering, -} - -impl MinMaxStatistics { - /// Sort order used to sort the statistics - #[allow(unused)] - pub fn sort_order(&self) -> &LexOrdering { - &self.sort_order - } - - /// Min value at index - #[allow(unused)] - pub fn min(&self, idx: usize) -> Row { - self.min_by_sort_order.row(idx) - } - - /// Max value at index - pub fn max(&self, idx: usize) -> Row { - self.max_by_sort_order.row(idx) - } - - pub fn new_from_files<'a>( - projected_sort_order: &LexOrdering, // Sort order with respect to projected schema - projected_schema: &SchemaRef, // Projected schema - projection: Option<&[usize]>, // Indices of projection in full table schema (None = all columns) - files: impl IntoIterator, - ) -> Result { - use datafusion_common::ScalarValue; - - let statistics_and_partition_values = files - .into_iter() - .map(|file| { - file.statistics - .as_ref() - .zip(Some(file.partition_values.as_slice())) - }) - .collect::>>() - .ok_or_else(|| { - DataFusionError::Plan("Parquet file missing statistics".to_string()) - })?; - - // Helper function to get min/max statistics for a given column of projected_schema - let get_min_max = |i: usize| -> Result<(Vec, Vec)> { - Ok(statistics_and_partition_values - .iter() - .map(|(s, pv)| { - if i < s.column_statistics.len() { - s.column_statistics[i] - .min_value - .get_value() - .cloned() - .zip(s.column_statistics[i].max_value.get_value().cloned()) - .ok_or_else(|| { - DataFusionError::Plan("statistics not found".to_string()) - }) - } else { - let partition_value = &pv[i - s.column_statistics.len()]; - Ok((partition_value.clone(), partition_value.clone())) - } - }) - .collect::>>()? - .into_iter() - .unzip()) - }; - - let sort_columns = sort_columns_from_physical_sort_exprs(projected_sort_order) - .ok_or(DataFusionError::Plan( - "sort expression must be on column".to_string(), - ))?; - - // Project the schema & sort order down to just the relevant columns - let min_max_schema = Arc::new( - projected_schema - .project(&(sort_columns.iter().map(|c| c.index()).collect::>()))?, - ); - let min_max_sort_order = LexOrdering::from( - sort_columns - .iter() - .zip(projected_sort_order.iter()) - .enumerate() - .map(|(i, (col, sort))| PhysicalSortExpr { - expr: Arc::new(Column::new(col.name(), i)), - options: sort.options, - }) - .collect::>(), - ); - - let (min_values, max_values): (Vec<_>, Vec<_>) = sort_columns - .iter() - .map(|c| { - // Reverse the projection to get the index of the column in the full statistics - // The file statistics contains _every_ column , but the sort column's index() - // refers to the index in projected_schema - let i = projection.map(|p| p[c.index()]).unwrap_or(c.index()); - - let (min, max) = get_min_max(i).map_err(|e| { - e.context(format!("get min/max for column: '{}'", c.name())) - })?; - Ok(( - ScalarValue::iter_to_array(min)?, - ScalarValue::iter_to_array(max)?, - )) - }) - .collect::>>() - .map_err(|e| e.context("collect min/max values"))? - .into_iter() - .unzip(); - - Self::new( - &min_max_sort_order, - &min_max_schema, - RecordBatch::try_new(Arc::clone(&min_max_schema), min_values).map_err( - |e| { - DataFusionError::ArrowError(e, Some("\ncreate min batch".to_string())) - }, - )?, - RecordBatch::try_new(Arc::clone(&min_max_schema), max_values).map_err( - |e| { - DataFusionError::ArrowError(e, Some("\ncreate max batch".to_string())) - }, - )?, - ) - } - - pub fn new( - sort_order: &LexOrdering, - schema: &SchemaRef, - min_values: RecordBatch, - max_values: RecordBatch, - ) -> Result { - use arrow::row::*; - - let sort_fields = sort_order - .iter() - .map(|expr| { - expr.expr - .data_type(schema) - .map(|data_type| SortField::new_with_options(data_type, expr.options)) - }) - .collect::>>() - .map_err(|e| e.context("create sort fields"))?; - let converter = RowConverter::new(sort_fields)?; - - let sort_columns = sort_columns_from_physical_sort_exprs(sort_order).ok_or( - DataFusionError::Plan("sort expression must be on column".to_string()), - )?; - - // swap min/max if they're reversed in the ordering - let (new_min_cols, new_max_cols): (Vec<_>, Vec<_>) = sort_order - .iter() - .zip(sort_columns.iter().copied()) - .map(|(sort_expr, column)| { - if sort_expr.options.descending { - max_values - .column_by_name(column.name()) - .zip(min_values.column_by_name(column.name())) - } else { - min_values - .column_by_name(column.name()) - .zip(max_values.column_by_name(column.name())) - } - .ok_or_else(|| { - DataFusionError::Plan(format!( - "missing column in MinMaxStatistics::new: '{}'", - column.name() - )) - }) - }) - .collect::>>()? - .into_iter() - .unzip(); - - let [min, max] = [new_min_cols, new_max_cols].map(|cols| { - let values = RecordBatch::try_new( - min_values.schema(), - cols.into_iter().cloned().collect(), - )?; - let sorting_columns = sort_order - .iter() - .zip(sort_columns.iter().copied()) - .map(|(sort_expr, column)| { - let schema = values.schema(); - - let idx = schema.index_of(column.name())?; - let field = schema.field(idx); - - // check that sort columns are non-nullable - if field.is_nullable() { - return plan_err!("cannot sort by nullable column"); - } - - Ok(SortColumn { - values: Arc::clone(values.column(idx)), - options: Some(sort_expr.options), - }) - }) - .collect::>>() - .map_err(|e| e.context("create sorting columns"))?; - converter - .convert_columns( - &sorting_columns - .into_iter() - .map(|c| c.values) - .collect::>(), - ) - .map_err(|e| { - DataFusionError::ArrowError(e, Some("convert columns".to_string())) - }) - }); - - Ok(Self { - min_by_sort_order: min.map_err(|e| e.context("build min rows"))?, - max_by_sort_order: max.map_err(|e| e.context("build max rows"))?, - sort_order: sort_order.clone(), - }) - } - - /// Return a sorted list of the min statistics together with the original indices - pub fn min_values_sorted(&self) -> Vec<(usize, Row<'_>)> { - let mut sort: Vec<_> = self.min_by_sort_order.iter().enumerate().collect(); - sort.sort_unstable_by(|(_, a), (_, b)| a.cmp(b)); - sort - } - - /// Check if the min/max statistics are in order and non-overlapping - pub fn is_sorted(&self) -> bool { - self.max_by_sort_order - .iter() - .zip(self.min_by_sort_order.iter().skip(1)) - .all(|(max, next_min)| max < next_min) - } -} - -fn sort_columns_from_physical_sort_exprs( - sort_order: &LexOrdering, -) -> Option> { - sort_order - .iter() - .map(|expr| expr.expr.as_any().downcast_ref::()) - .collect::>>() -}