From 9285b84ce90180eab96241939ef9e97632f35566 Mon Sep 17 00:00:00 2001 From: Adam Gutglick Date: Tue, 25 Feb 2025 13:45:22 +0000 Subject: [PATCH] Move `FileSourceConfig` and `FileStream` to the new `datafusion-datasource` (#14838) * Initial work * Fix some CI issues * remove cyclical dev-dependency on core * Trying to keep some key things accessible in the same way * ignore rustdoc test for example * Restore doc test with mock parquet source --------- Co-authored-by: Andrew Lamb --- .../examples/csv_json_opener.rs | 2 +- .../examples/custom_file_format.rs | 5 +- datafusion/common/src/test_util.rs | 2 +- datafusion/core/Cargo.toml | 2 +- .../core/src/datasource/file_format/arrow.rs | 8 +- .../core/src/datasource/file_format/avro.rs | 5 +- .../core/src/datasource/file_format/csv.rs | 8 +- .../core/src/datasource/file_format/json.rs | 7 +- .../core/src/datasource/file_format/mod.rs | 5 +- .../src/datasource/file_format/parquet.rs | 8 +- .../core/src/datasource/listing/table.rs | 3 +- datafusion/core/src/datasource/mod.rs | 1 - .../datasource/physical_plan/arrow_file.rs | 5 +- .../core/src/datasource/physical_plan/avro.rs | 6 +- .../core/src/datasource/physical_plan/csv.rs | 6 +- .../physical_plan/file_scan_config.rs | 1244 -------------- .../datasource/physical_plan/file_stream.rs | 799 --------- .../core/src/datasource/physical_plan/json.rs | 6 +- .../core/src/datasource/physical_plan/mod.rs | 462 +----- .../datasource/physical_plan/parquet/mod.rs | 5 +- .../physical_plan/parquet/source.rs | 7 +- .../core/src/datasource/schema_adapter.rs | 3 +- datafusion/core/src/test/mod.rs | 5 +- datafusion/core/src/test_util/parquet.rs | 3 +- .../physical_optimizer/enforce_sorting.rs | 3 +- .../tests/physical_optimizer/test_utils.rs | 3 +- datafusion/datasource/Cargo.toml | 2 + datafusion/datasource/src/display.rs | 295 ++++ .../data_source.rs => datasource/src/file.rs} | 8 +- datafusion/datasource/src/file_scan_config.rs | 1447 ++++++++++++++++- datafusion/datasource/src/file_stream.rs | 773 ++++++++- datafusion/datasource/src/memory.rs | 40 +- datafusion/datasource/src/mod.rs | 45 +- .../src}/statistics.rs | 14 +- datafusion/datasource/src/test_util.rs | 84 + .../proto/src/physical_plan/from_proto.rs | 3 +- 36 files changed, 2727 insertions(+), 2597 deletions(-) delete mode 100644 datafusion/core/src/datasource/physical_plan/file_scan_config.rs delete mode 100644 datafusion/core/src/datasource/physical_plan/file_stream.rs create mode 100644 datafusion/datasource/src/display.rs rename datafusion/{core/src/datasource/data_source.rs => datasource/src/file.rs} (95%) rename datafusion/{core/src/datasource/physical_plan => datasource/src}/statistics.rs (97%) create mode 100644 datafusion/datasource/src/test_util.rs diff --git a/datafusion-examples/examples/csv_json_opener.rs b/datafusion-examples/examples/csv_json_opener.rs index ef4ff9d51e7f..574137afe5c9 100644 --- a/datafusion-examples/examples/csv_json_opener.rs +++ b/datafusion-examples/examples/csv_json_opener.rs @@ -21,8 +21,8 @@ use arrow::datatypes::{DataType, Field, Schema}; use datafusion::datasource::physical_plan::JsonSource; use datafusion::{ assert_batches_eq, + datasource::physical_plan::FileSource, datasource::{ - data_source::FileSource, file_format::file_compression_type::FileCompressionType, listing::PartitionedFile, object_store::ObjectStoreUrl, diff --git a/datafusion-examples/examples/custom_file_format.rs b/datafusion-examples/examples/custom_file_format.rs index c44210e55318..165d82627061 100644 --- a/datafusion-examples/examples/custom_file_format.rs +++ b/datafusion-examples/examples/custom_file_format.rs @@ -21,14 +21,15 @@ use arrow::{ array::{AsArray, RecordBatch, StringArray, UInt8Array}, datatypes::{DataType, Field, Schema, SchemaRef, UInt64Type}, }; -use datafusion::datasource::data_source::FileSource; -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::physical_plan::FileSource, execution::session_state::SessionStateBuilder, +}; use datafusion::{ datasource::{ file_format::{ diff --git a/datafusion/common/src/test_util.rs b/datafusion/common/src/test_util.rs index a1f883f20525..298f54389cf8 100644 --- a/datafusion/common/src/test_util.rs +++ b/datafusion/common/src/test_util.rs @@ -28,7 +28,7 @@ use std::{error::Error, path::PathBuf}; /// /// Expects to be called about like this: /// -/// `assert_batch_eq!(expected_lines: &[&str], batches: &[RecordBatch])` +/// `assert_batches_eq!(expected_lines: &[&str], batches: &[RecordBatch])` /// /// # Example /// ``` diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index 1e0f27ccdfc8..69048f6a7cf4 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -40,7 +40,7 @@ nested_expressions = ["datafusion-functions-nested"] # This feature is deprecated. Use the `nested_expressions` feature instead. array_expressions = ["nested_expressions"] # Used to enable the avro format -avro = ["apache-avro", "num-traits", "datafusion-common/avro"] +avro = ["apache-avro", "num-traits", "datafusion-common/avro", "datafusion-datasource/avro"] backtrace = ["datafusion-common/backtrace"] compression = ["xz2", "bzip2", "flate2", "zstd", "datafusion-datasource/compression"] crypto_expressions = ["datafusion-functions/crypto_expressions"] diff --git a/datafusion/core/src/datasource/file_format/arrow.rs b/datafusion/core/src/datasource/file_format/arrow.rs index 09121eba6702..3614b788af90 100644 --- a/datafusion/core/src/datasource/file_format/arrow.rs +++ b/datafusion/core/src/datasource/file_format/arrow.rs @@ -31,9 +31,7 @@ use super::write::{create_writer, SharedBuffer}; use super::FileFormatFactory; use crate::datasource::file_format::write::get_writer_schema; use crate::datasource::file_format::FileFormat; -use crate::datasource::physical_plan::{ - ArrowSource, FileGroupDisplay, FileScanConfig, FileSink, FileSinkConfig, -}; +use crate::datasource::physical_plan::{ArrowSource, FileSink, FileSinkConfig}; use crate::error::Result; use crate::physical_plan::{DisplayAs, DisplayFormatType, ExecutionPlan}; @@ -49,13 +47,15 @@ use datafusion_common::{ not_impl_err, DataFusionError, GetExt, Statistics, DEFAULT_ARROW_EXTENSION, }; use datafusion_common_runtime::SpawnedTask; +use datafusion_datasource::display::FileGroupDisplay; +use datafusion_datasource::file::FileSource; +use datafusion_datasource::file_scan_config::FileScanConfig; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_expr::dml::InsertOp; use datafusion_physical_expr::PhysicalExpr; use datafusion_physical_expr_common::sort_expr::LexRequirement; use datafusion_physical_plan::insert::{DataSink, DataSinkExec}; -use crate::datasource::data_source::FileSource; use async_trait::async_trait; use bytes::Bytes; use futures::stream::BoxStream; diff --git a/datafusion/core/src/datasource/file_format/avro.rs b/datafusion/core/src/datasource/file_format/avro.rs index c0c8f25722c2..e7314e839bf2 100644 --- a/datafusion/core/src/datasource/file_format/avro.rs +++ b/datafusion/core/src/datasource/file_format/avro.rs @@ -26,12 +26,11 @@ use super::file_compression_type::FileCompressionType; use super::FileFormat; use super::FileFormatFactory; use crate::datasource::avro_to_arrow::read_avro_schema_from_reader; -use crate::datasource::physical_plan::{AvroSource, FileScanConfig}; +use crate::datasource::physical_plan::AvroSource; use crate::error::Result; use crate::physical_plan::ExecutionPlan; use crate::physical_plan::Statistics; -use crate::datasource::data_source::FileSource; use arrow::datatypes::Schema; use arrow::datatypes::SchemaRef; use async_trait::async_trait; @@ -40,6 +39,8 @@ use datafusion_common::internal_err; use datafusion_common::parsers::CompressionTypeVariant; use datafusion_common::GetExt; use datafusion_common::DEFAULT_AVRO_EXTENSION; +use datafusion_datasource::file::FileSource; +use datafusion_datasource::file_scan_config::FileScanConfig; use datafusion_physical_expr::PhysicalExpr; use object_store::{GetResultPayload, ObjectMeta, ObjectStore}; diff --git a/datafusion/core/src/datasource/file_format/csv.rs b/datafusion/core/src/datasource/file_format/csv.rs index 4991a96dc3d3..45ad3e8c1c30 100644 --- a/datafusion/core/src/datasource/file_format/csv.rs +++ b/datafusion/core/src/datasource/file_format/csv.rs @@ -30,9 +30,7 @@ use super::{ use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::file_format::write::demux::DemuxedStreamReceiver; use crate::datasource::file_format::write::BatchSerializer; -use crate::datasource::physical_plan::{ - CsvSource, FileGroupDisplay, FileScanConfig, FileSink, FileSinkConfig, -}; +use crate::datasource::physical_plan::{CsvSource, FileSink, FileSinkConfig}; use crate::error::Result; use crate::execution::context::SessionState; use crate::physical_plan::insert::{DataSink, DataSinkExec}; @@ -51,12 +49,14 @@ use datafusion_common::{ exec_err, not_impl_err, DataFusionError, GetExt, DEFAULT_CSV_EXTENSION, }; use datafusion_common_runtime::SpawnedTask; +use datafusion_datasource::display::FileGroupDisplay; +use datafusion_datasource::file::FileSource; +use datafusion_datasource::file_scan_config::FileScanConfig; use datafusion_execution::TaskContext; use datafusion_expr::dml::InsertOp; use datafusion_physical_expr::PhysicalExpr; use datafusion_physical_expr_common::sort_expr::LexRequirement; -use crate::datasource::data_source::FileSource; use async_trait::async_trait; use bytes::{Buf, Bytes}; use futures::stream::BoxStream; diff --git a/datafusion/core/src/datasource/file_format/json.rs b/datafusion/core/src/datasource/file_format/json.rs index 94e74b144499..7a5aaf7c64e8 100644 --- a/datafusion/core/src/datasource/file_format/json.rs +++ b/datafusion/core/src/datasource/file_format/json.rs @@ -32,9 +32,7 @@ use super::{ use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::file_format::write::demux::DemuxedStreamReceiver; use crate::datasource::file_format::write::BatchSerializer; -use crate::datasource::physical_plan::{ - FileGroupDisplay, FileSink, FileSinkConfig, JsonSource, -}; +use crate::datasource::physical_plan::{FileSink, FileSinkConfig, JsonSource}; use crate::error::Result; use crate::execution::SessionState; use crate::physical_plan::insert::{DataSink, DataSinkExec}; @@ -52,12 +50,13 @@ 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}; use datafusion_common_runtime::SpawnedTask; +use datafusion_datasource::display::FileGroupDisplay; +use datafusion_datasource::file::FileSource; use datafusion_execution::TaskContext; use datafusion_expr::dml::InsertOp; use datafusion_physical_expr::PhysicalExpr; use datafusion_physical_plan::ExecutionPlan; -use crate::datasource::data_source::FileSource; use async_trait::async_trait; use bytes::{Buf, Bytes}; use datafusion_physical_expr_common::sort_expr::LexRequirement; diff --git a/datafusion/core/src/datasource/file_format/mod.rs b/datafusion/core/src/datasource/file_format/mod.rs index 657fe6ca5511..2b46748d0a52 100644 --- a/datafusion/core/src/datasource/file_format/mod.rs +++ b/datafusion/core/src/datasource/file_format/mod.rs @@ -28,7 +28,9 @@ pub mod json; pub mod options; #[cfg(feature = "parquet")] pub mod parquet; +use datafusion_datasource::file::FileSource; pub use datafusion_datasource::file_compression_type; +use datafusion_datasource::file_scan_config::FileScanConfig; pub use datafusion_datasource::write; use std::any::Any; @@ -40,7 +42,7 @@ use std::task::Poll; use crate::arrow::array::RecordBatch; use crate::arrow::datatypes::{DataType, Field, FieldRef, Schema, SchemaRef}; use crate::arrow::error::ArrowError; -use crate::datasource::physical_plan::{FileScanConfig, FileSinkConfig}; +use crate::datasource::physical_plan::FileSinkConfig; use crate::error::Result; use crate::physical_plan::{ExecutionPlan, Statistics}; @@ -50,7 +52,6 @@ use datafusion_common::{internal_err, not_impl_err, GetExt}; use datafusion_expr::Expr; use datafusion_physical_expr::PhysicalExpr; -use crate::datasource::data_source::FileSource; use async_trait::async_trait; use bytes::{Buf, Bytes}; use datafusion_physical_expr_common::sort_expr::LexRequirement; diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index 7dbc510eca09..e9ecff7baff5 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -28,7 +28,7 @@ use super::write::{create_writer, SharedBuffer}; use super::{ coerce_file_schema_to_string_type, coerce_file_schema_to_view_type, transform_binary_to_string, transform_schema_to_view, FileFormat, FileFormatFactory, - FilePushdownSupport, FileScanConfig, + FilePushdownSupport, }; use crate::arrow::array::RecordBatch; use crate::arrow::datatypes::{Fields, Schema, SchemaRef}; @@ -36,7 +36,7 @@ use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::file_format::write::get_writer_schema; use crate::datasource::physical_plan::parquet::can_expr_be_pushed_down_with_schemas; use crate::datasource::physical_plan::parquet::source::ParquetSource; -use crate::datasource::physical_plan::{FileGroupDisplay, FileSink, FileSinkConfig}; +use crate::datasource::physical_plan::{FileSink, FileSinkConfig}; use crate::datasource::statistics::{create_max_min_accs, get_col_stats}; use crate::error::Result; use crate::execution::SessionState; @@ -57,6 +57,9 @@ use datafusion_common::{ DEFAULT_PARQUET_EXTENSION, }; use datafusion_common_runtime::SpawnedTask; +use datafusion_datasource::display::FileGroupDisplay; +use datafusion_datasource::file::FileSource; +use datafusion_datasource::file_scan_config::FileScanConfig; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryPool, MemoryReservation}; use datafusion_execution::TaskContext; use datafusion_expr::dml::InsertOp; @@ -65,7 +68,6 @@ use datafusion_functions_aggregate::min_max::{MaxAccumulator, MinAccumulator}; use datafusion_physical_expr::PhysicalExpr; use datafusion_physical_expr_common::sort_expr::LexRequirement; -use crate::datasource::data_source::FileSource; use async_trait::async_trait; use bytes::Bytes; use futures::future::BoxFuture; diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 819da155a1a2..adef02c38d73 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -29,11 +29,12 @@ use crate::datasource::{ file_compression_type::FileCompressionType, FileFormat, FilePushdownSupport, }, get_statistics_with_limit, - physical_plan::{FileScanConfig, FileSinkConfig}, + physical_plan::FileSinkConfig, }; use crate::execution::context::SessionState; use datafusion_catalog::TableProvider; use datafusion_common::{config_err, DataFusionError, Result}; +use datafusion_datasource::file_scan_config::FileScanConfig; use datafusion_expr::dml::InsertOp; use datafusion_expr::{utils::conjunction, Expr, TableProviderFilterPushDown}; use datafusion_expr::{SortExpr, TableType}; diff --git a/datafusion/core/src/datasource/mod.rs b/datafusion/core/src/datasource/mod.rs index 96687913fb42..2b7bb14b6f6c 100644 --- a/datafusion/core/src/datasource/mod.rs +++ b/datafusion/core/src/datasource/mod.rs @@ -21,7 +21,6 @@ pub mod avro_to_arrow; pub mod cte_worktable; -pub mod data_source; pub mod default_table_source; pub mod dynamic_file; pub mod empty; diff --git a/datafusion/core/src/datasource/physical_plan/arrow_file.rs b/datafusion/core/src/datasource/physical_plan/arrow_file.rs index e5523063c782..1cae5c5084b1 100644 --- a/datafusion/core/src/datasource/physical_plan/arrow_file.rs +++ b/datafusion/core/src/datasource/physical_plan/arrow_file.rs @@ -20,10 +20,9 @@ use std::any::Any; use std::sync::Arc; -use crate::datasource::data_source::FileSource; use crate::datasource::listing::PartitionedFile; use crate::datasource::physical_plan::{ - FileMeta, FileOpenFuture, FileOpener, FileScanConfig, JsonSource, + FileMeta, FileOpenFuture, FileOpener, JsonSource, }; use crate::error::Result; @@ -32,6 +31,8 @@ use arrow::datatypes::SchemaRef; use arrow_ipc::reader::FileDecoder; use datafusion_common::config::ConfigOptions; use datafusion_common::{Constraints, Statistics}; +use datafusion_datasource::file::FileSource; +use datafusion_datasource::file_scan_config::FileScanConfig; use datafusion_datasource::source::DataSourceExec; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_physical_expr::{EquivalenceProperties, Partitioning}; diff --git a/datafusion/core/src/datasource/physical_plan/avro.rs b/datafusion/core/src/datasource/physical_plan/avro.rs index 1674814d76a7..08c22183302b 100644 --- a/datafusion/core/src/datasource/physical_plan/avro.rs +++ b/datafusion/core/src/datasource/physical_plan/avro.rs @@ -21,14 +21,16 @@ use std::any::Any; use std::fmt::Formatter; use std::sync::Arc; -use super::{FileOpener, FileScanConfig}; +use super::FileOpener; #[cfg(feature = "avro")] use crate::datasource::avro_to_arrow::Reader as AvroReader; -use crate::datasource::data_source::FileSource; + use crate::error::Result; use arrow::datatypes::SchemaRef; use datafusion_common::{Constraints, Statistics}; +use datafusion_datasource::file::FileSource; +use datafusion_datasource::file_scan_config::FileScanConfig; use datafusion_datasource::source::DataSourceExec; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_physical_expr::{EquivalenceProperties, Partitioning}; diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index 629d452064f5..1552060d067d 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -23,8 +23,8 @@ use std::io::{Read, Seek, SeekFrom}; use std::sync::Arc; use std::task::Poll; -use super::{calculate_range, FileScanConfig, RangeCalculation}; -use crate::datasource::data_source::FileSource; +use super::{calculate_range, RangeCalculation}; + use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::file_format::{deserialize_stream, DecoderDeserializer}; use crate::datasource::listing::{FileRange, ListingTableUrl, PartitionedFile}; @@ -37,6 +37,8 @@ use arrow::csv; use arrow::datatypes::SchemaRef; use datafusion_common::config::ConfigOptions; use datafusion_common::{Constraints, Statistics}; +use datafusion_datasource::file::FileSource; +use datafusion_datasource::file_scan_config::FileScanConfig; use datafusion_datasource::source::DataSourceExec; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_physical_expr::{EquivalenceProperties, Partitioning}; 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 6b74f6be79eb..000000000000 --- a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs +++ /dev/null @@ -1,1244 +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 super::{ - get_projected_output_ordering, statistics::MinMaxStatistics, FileGroupsDisplay, - FileStream, -}; -use crate::datasource::file_format::file_compression_type::FileCompressionType; -use crate::datasource::{listing::PartitionedFile, object_store::ObjectStoreUrl}; -use crate::{error::Result, scalar::ScalarValue}; -use std::any::Any; -use std::fmt::Formatter; -use std::{fmt, sync::Arc}; - -use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; -use datafusion_common::stats::Precision; -use datafusion_common::{ColumnStatistics, Constraints, Statistics}; -use datafusion_physical_expr::{EquivalenceProperties, LexOrdering, Partitioning}; - -use crate::datasource::data_source::FileSource; -pub use datafusion_datasource::file_scan_config::*; -use datafusion_datasource::source::{DataSource, DataSourceExec}; -use datafusion_execution::{SendableRecordBatchStream, TaskContext}; -use datafusion_physical_plan::display::{display_orderings, ProjectSchemaDisplay}; -use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; -use datafusion_physical_plan::projection::{ - all_alias_free_columns, new_projections_for_columns, ProjectionExec, -}; -use datafusion_physical_plan::{DisplayAs, DisplayFormatType, ExecutionPlan}; - -/// 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 for a [`DataSourceExec`], the a physical plan for -/// any given file format. -/// -/// Use [`Self::build`] to create a [`DataSourceExec`] from a ``FileScanConfig`. -/// -/// # Example -/// ``` -/// # use std::sync::Arc; -/// # use arrow::datatypes::{Field, Fields, DataType, Schema}; -/// # use datafusion::datasource::listing::PartitionedFile; -/// # use datafusion::datasource::physical_plan::FileScanConfig; -/// # use datafusion_execution::object_store::ObjectStoreUrl; -/// # use datafusion::datasource::physical_plan::ArrowSource; -/// # use datafusion_physical_plan::ExecutionPlan; -/// # let file_schema = Arc::new(Schema::new(vec![ -/// # Field::new("c1", DataType::Int32, false), -/// # Field::new("c2", DataType::Int32, false), -/// # Field::new("c3", DataType::Int32, false), -/// # Field::new("c4", DataType::Int32, false), -/// # ])); -/// // create FileScan config for reading arrow files from file:// -/// let object_store_url = ObjectStoreUrl::local_filesystem(); -/// let file_source = Arc::new(ArrowSource::default()); -/// let config = FileScanConfig::new(object_store_url, file_schema, file_source) -/// .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), -/// ]); -/// // create an execution plan from the config -/// let plan: Arc = config.build(); -/// ``` -#[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, - /// File compression type - pub file_compression_type: FileCompressionType, - /// Are new lines in values supported for CSVOptions - pub new_lines_in_values: bool, - /// File source such as `ParquetSource`, `CsvSource`, `JsonSource`, etc. - pub source: Arc, -} - -impl DataSource for FileScanConfig { - fn open( - &self, - partition: usize, - context: Arc, - ) -> Result { - let object_store = context.runtime_env().object_store(&self.object_store_url)?; - - let source = self - .source - .with_batch_size(context.session_config().batch_size()) - .with_schema(Arc::clone(&self.file_schema)) - .with_projection(self); - - let opener = source.create_file_opener(object_store, self, partition); - - let stream = FileStream::new(self, partition, opener, source.metrics())?; - Ok(Box::pin(stream)) - } - - fn as_any(&self) -> &dyn Any { - self - } - - fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> fmt::Result { - 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)?; - } - - self.fmt_file_source(t, f) - } - - /// If supported by the underlying [`FileSource`], redistribute files across partitions according to their size. - fn repartitioned( - &self, - target_partitions: usize, - repartition_file_min_size: usize, - output_ordering: Option, - ) -> Result>> { - let source = self.source.repartitioned( - target_partitions, - repartition_file_min_size, - output_ordering, - self, - )?; - - Ok(source.map(|s| Arc::new(s) as _)) - } - - fn output_partitioning(&self) -> Partitioning { - Partitioning::UnknownPartitioning(self.file_groups.len()) - } - - fn eq_properties(&self) -> EquivalenceProperties { - let (schema, constraints, _, orderings) = self.project(); - EquivalenceProperties::new_with_orderings(schema, orderings.as_slice()) - .with_constraints(constraints) - } - - fn statistics(&self) -> Result { - self.source.statistics() - } - - fn with_fetch(&self, limit: Option) -> Option> { - let source = self.clone(); - Some(Arc::new(source.with_limit(limit))) - } - - fn fetch(&self) -> Option { - self.limit - } - - fn metrics(&self) -> ExecutionPlanMetricsSet { - self.source.metrics().clone() - } - - fn try_swapping_with_projection( - &self, - projection: &ProjectionExec, - ) -> Result>> { - // If there is any non-column or alias-carrier expression, Projection should not be removed. - // This process can be moved into CsvExec, but it would be an overlap of their responsibility. - Ok(all_alias_free_columns(projection.expr()).then(|| { - let file_scan = self.clone(); - let source = Arc::clone(&file_scan.source); - let new_projections = new_projections_for_columns( - projection, - &file_scan - .projection - .clone() - .unwrap_or((0..self.file_schema.fields().len()).collect()), - ); - file_scan - // Assign projected statistics to source - .with_projection(Some(new_projections)) - .with_source(source) - .build() as _ - })) - } -} - -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, - file_source: Arc, - ) -> Self { - let statistics = Statistics::new_unknown(&file_schema); - - let mut config = Self { - object_store_url, - file_schema, - file_groups: vec![], - constraints: Constraints::empty(), - statistics, - projection: None, - limit: None, - table_partition_cols: vec![], - output_ordering: vec![], - file_compression_type: FileCompressionType::UNCOMPRESSED, - new_lines_in_values: false, - source: Arc::clone(&file_source), - }; - - config = config.with_source(Arc::clone(&file_source)); - config - } - - /// Set the file source - pub fn with_source(mut self, source: Arc) -> Self { - let ( - _projected_schema, - _constraints, - projected_statistics, - _projected_output_ordering, - ) = self.project(); - self.source = source.with_statistics(projected_statistics); - self - } - - /// 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 - } - - /// Set the file compression type - pub fn with_file_compression_type( - mut self, - file_compression_type: FileCompressionType, - ) -> Self { - self.file_compression_type = file_compression_type; - self - } - - /// Set the new_lines_in_values property - pub fn with_newlines_in_values(mut self, new_lines_in_values: bool) -> Self { - self.new_lines_in_values = new_lines_in_values; - self - } - - /// Specifies whether newlines in (quoted) values are supported. - /// - /// Parsing newlines in quoted values may be affected by execution behaviour such as - /// parallel file scanning. Setting this to `true` ensures that newlines in values are - /// parsed successfully, which may reduce performance. - /// - /// The default behaviour depends on the `datafusion.catalog.newlines_in_values` setting. - pub fn newlines_in_values(&self) -> bool { - self.new_lines_in_values - } - - /// 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()) - } - - // TODO: This function should be moved into DataSourceExec once FileScanConfig moved out of datafusion/core - /// Returns a new [`DataSourceExec`] to scan the files specified by this config - pub fn build(self) -> Arc { - Arc::new(DataSourceExec::new(Arc::new(self))) - } - - /// Write the data_type based on file_source - fn fmt_file_source(&self, t: DisplayFormatType, f: &mut Formatter) -> fmt::Result { - write!(f, ", file_type={}", self.source.file_type())?; - self.source.fmt_extra(t, f) - } - - /// Returns the file_source - pub fn file_source(&self) -> &Arc { - &self.source - } -} - -#[cfg(test)] -mod tests { - use super::*; - use crate::datasource::physical_plan::ArrowSource; - use crate::{test::columns, test_util::aggr_test_schema}; - use arrow::array::{Int32Array, RecordBatch}; - use std::collections::HashMap; - - #[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, - Arc::new(ArrowSource::default()), - ) - .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/file_stream.rs b/datafusion/core/src/datasource/physical_plan/file_stream.rs deleted file mode 100644 index 7944d6fa9020..000000000000 --- a/datafusion/core/src/datasource/physical_plan/file_stream.rs +++ /dev/null @@ -1,799 +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. - -//! A generic stream over file format readers that can be used by -//! any file format that read its files from start to end. -//! -//! Note: Most traits here need to be marked `Sync + Send` to be -//! compliant with the `SendableRecordBatchStream` trait. - -use std::collections::VecDeque; -use std::mem; -use std::pin::Pin; -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, ExecutionPlanMetricsSet}; -use crate::physical_plan::RecordBatchStream; - -use arrow::datatypes::SchemaRef; -use arrow::error::ArrowError; -use arrow::record_batch::RecordBatch; -use datafusion_common::ScalarValue; -pub use datafusion_datasource::file_stream::{FileOpenFuture, FileOpener, OnError}; -use datafusion_datasource::file_stream::{FileStreamMetrics, FileStreamState, NextOpen}; - -use futures::{ready, FutureExt, Stream, StreamExt}; - -/// A stream that iterates record batch by record batch, file over file. -pub struct FileStream { - /// An iterator over input files. - file_iter: VecDeque, - /// The stream schema (file schema including partition columns and after - /// projection). - projected_schema: SchemaRef, - /// The remaining number of records to parse, None if no limit - remain: Option, - /// A dynamic [`FileOpener`]. Calling `open()` returns a [`FileOpenFuture`], - /// which can be resolved to a stream of `RecordBatch`. - file_opener: Arc, - /// The partition column projector - pc_projector: PartitionColumnProjector, - /// The stream state - state: FileStreamState, - /// File stream specific metrics - file_stream_metrics: FileStreamMetrics, - /// runtime baseline metrics - baseline_metrics: BaselineMetrics, - /// Describes the behavior of the `FileStream` if file opening or scanning fails - on_error: OnError, -} - -impl FileStream { - /// Create a new `FileStream` using the give `FileOpener` to scan underlying files - pub fn new( - config: &FileScanConfig, - partition: usize, - file_opener: Arc, - metrics: &ExecutionPlanMetricsSet, - ) -> Result { - let (projected_schema, ..) = config.project(); - let pc_projector = PartitionColumnProjector::new( - Arc::clone(&projected_schema), - &config - .table_partition_cols - .iter() - .map(|x| x.name().clone()) - .collect::>(), - ); - - let files = config.file_groups[partition].clone(); - - Ok(Self { - file_iter: files.into(), - projected_schema, - remain: config.limit, - file_opener, - pc_projector, - state: FileStreamState::Idle, - file_stream_metrics: FileStreamMetrics::new(metrics, partition), - baseline_metrics: BaselineMetrics::new(metrics, partition), - on_error: OnError::Fail, - }) - } - - /// Specify the behavior when an error occurs opening or scanning a file - /// - /// If `OnError::Skip` the stream will skip files which encounter an error and continue - /// If `OnError:Fail` (default) the stream will fail and stop processing when an error occurs - pub fn with_on_error(mut self, on_error: OnError) -> Self { - self.on_error = on_error; - self - } - - /// Begin opening the next file in parallel while decoding the current file in FileStream. - /// - /// Since file opening is mostly IO (and may involve a - /// bunch of sequential IO), it can be parallelized with decoding. - fn start_next_file(&mut self) -> Option)>> { - let part_file = self.file_iter.pop_front()?; - - let file_meta = FileMeta { - object_meta: part_file.object_meta, - range: part_file.range, - extensions: part_file.extensions, - metadata_size_hint: part_file.metadata_size_hint, - }; - - Some( - self.file_opener - .open(file_meta) - .map(|future| (future, part_file.partition_values)), - ) - } - - fn poll_inner(&mut self, cx: &mut Context<'_>) -> Poll>> { - loop { - match &mut self.state { - FileStreamState::Idle => { - self.file_stream_metrics.time_opening.start(); - - match self.start_next_file().transpose() { - Ok(Some((future, partition_values))) => { - self.state = FileStreamState::Open { - future, - partition_values, - } - } - Ok(None) => return Poll::Ready(None), - Err(e) => { - self.state = FileStreamState::Error; - return Poll::Ready(Some(Err(e))); - } - } - } - FileStreamState::Open { - future, - partition_values, - } => match ready!(future.poll_unpin(cx)) { - Ok(reader) => { - let partition_values = mem::take(partition_values); - - // include time needed to start opening in `start_next_file` - self.file_stream_metrics.time_opening.stop(); - let next = self.start_next_file().transpose(); - self.file_stream_metrics.time_scanning_until_data.start(); - self.file_stream_metrics.time_scanning_total.start(); - - match next { - Ok(Some((next_future, next_partition_values))) => { - self.state = FileStreamState::Scan { - partition_values, - reader, - next: Some(( - NextOpen::Pending(next_future), - next_partition_values, - )), - }; - } - Ok(None) => { - self.state = FileStreamState::Scan { - reader, - partition_values, - next: None, - }; - } - Err(e) => { - self.state = FileStreamState::Error; - return Poll::Ready(Some(Err(e))); - } - } - } - Err(e) => { - self.file_stream_metrics.file_open_errors.add(1); - match self.on_error { - OnError::Skip => { - self.file_stream_metrics.time_opening.stop(); - self.state = FileStreamState::Idle - } - OnError::Fail => { - self.state = FileStreamState::Error; - return Poll::Ready(Some(Err(e))); - } - } - } - }, - FileStreamState::Scan { - reader, - partition_values, - next, - } => { - // We need to poll the next `FileOpenFuture` here to drive it forward - if let Some((next_open_future, _)) = next { - if let NextOpen::Pending(f) = next_open_future { - if let Poll::Ready(reader) = f.as_mut().poll(cx) { - *next_open_future = NextOpen::Ready(reader); - } - } - } - match ready!(reader.poll_next_unpin(cx)) { - Some(Ok(batch)) => { - self.file_stream_metrics.time_scanning_until_data.stop(); - self.file_stream_metrics.time_scanning_total.stop(); - let result = self - .pc_projector - .project(batch, partition_values) - .map_err(|e| ArrowError::ExternalError(e.into())) - .map(|batch| match &mut self.remain { - Some(remain) => { - if *remain > batch.num_rows() { - *remain -= batch.num_rows(); - batch - } else { - let batch = batch.slice(0, *remain); - self.state = FileStreamState::Limit; - *remain = 0; - batch - } - } - None => batch, - }); - - if result.is_err() { - // If the partition value projection fails, this is not governed by - // the `OnError` behavior - self.state = FileStreamState::Error - } - self.file_stream_metrics.time_scanning_total.start(); - return Poll::Ready(Some(result.map_err(Into::into))); - } - Some(Err(err)) => { - self.file_stream_metrics.file_scan_errors.add(1); - self.file_stream_metrics.time_scanning_until_data.stop(); - self.file_stream_metrics.time_scanning_total.stop(); - - match self.on_error { - // If `OnError::Skip` we skip the file as soon as we hit the first error - OnError::Skip => match mem::take(next) { - Some((future, partition_values)) => { - self.file_stream_metrics.time_opening.start(); - - match future { - NextOpen::Pending(future) => { - self.state = FileStreamState::Open { - future, - partition_values, - } - } - NextOpen::Ready(reader) => { - self.state = FileStreamState::Open { - future: Box::pin(std::future::ready( - reader, - )), - partition_values, - } - } - } - } - None => return Poll::Ready(None), - }, - OnError::Fail => { - self.state = FileStreamState::Error; - return Poll::Ready(Some(Err(err.into()))); - } - } - } - None => { - self.file_stream_metrics.time_scanning_until_data.stop(); - self.file_stream_metrics.time_scanning_total.stop(); - - match mem::take(next) { - Some((future, partition_values)) => { - self.file_stream_metrics.time_opening.start(); - - match future { - NextOpen::Pending(future) => { - self.state = FileStreamState::Open { - future, - partition_values, - } - } - NextOpen::Ready(reader) => { - self.state = FileStreamState::Open { - future: Box::pin(std::future::ready( - reader, - )), - partition_values, - } - } - } - } - None => return Poll::Ready(None), - } - } - } - } - FileStreamState::Error | FileStreamState::Limit => { - return Poll::Ready(None) - } - } - } - } -} - -impl Stream for FileStream { - type Item = Result; - - fn poll_next( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { - self.file_stream_metrics.time_processing.start(); - let result = self.poll_inner(cx); - self.file_stream_metrics.time_processing.stop(); - self.baseline_metrics.record_poll(result) - } -} - -impl RecordBatchStream for FileStream { - fn schema(&self) -> SchemaRef { - Arc::clone(&self.projected_schema) - } -} - -#[cfg(test)] -mod tests { - use std::sync::atomic::{AtomicUsize, Ordering}; - use std::sync::Arc; - - use super::*; - use crate::datasource::object_store::ObjectStoreUrl; - use crate::prelude::SessionContext; - use crate::test::{make_partition, object_store::register_test_store}; - - use crate::datasource::physical_plan::CsvSource; - use arrow::datatypes::Schema; - use datafusion_common::internal_err; - - /// Test `FileOpener` which will simulate errors during file opening or scanning - #[derive(Default)] - struct TestOpener { - /// Index in stream of files which should throw an error while opening - error_opening_idx: Vec, - /// Index in stream of files which should throw an error while scanning - error_scanning_idx: Vec, - /// Index of last file in stream - current_idx: AtomicUsize, - /// `RecordBatch` to return - records: Vec, - } - - impl FileOpener for TestOpener { - fn open(&self, _file_meta: FileMeta) -> Result { - let idx = self.current_idx.fetch_add(1, Ordering::SeqCst); - - if self.error_opening_idx.contains(&idx) { - Ok(futures::future::ready(internal_err!("error opening")).boxed()) - } else if self.error_scanning_idx.contains(&idx) { - let error = futures::future::ready(Err(ArrowError::IpcError( - "error scanning".to_owned(), - ))); - let stream = futures::stream::once(error).boxed(); - Ok(futures::future::ready(Ok(stream)).boxed()) - } else { - let iterator = self.records.clone().into_iter().map(Ok); - let stream = futures::stream::iter(iterator).boxed(); - Ok(futures::future::ready(Ok(stream)).boxed()) - } - } - } - - #[derive(Default)] - struct FileStreamTest { - /// Number of files in the stream - num_files: usize, - /// Global limit of records emitted by the stream - limit: Option, - /// Error-handling behavior of the stream - on_error: OnError, - /// Mock `FileOpener` - opener: TestOpener, - } - - impl FileStreamTest { - pub fn new() -> Self { - Self::default() - } - - /// Specify the number of files in the stream - pub fn with_num_files(mut self, num_files: usize) -> Self { - self.num_files = num_files; - self - } - - /// Specify the limit - pub fn with_limit(mut self, limit: Option) -> Self { - self.limit = limit; - self - } - - /// Specify the index of files in the stream which should - /// throw an error when opening - pub fn with_open_errors(mut self, idx: Vec) -> Self { - self.opener.error_opening_idx = idx; - self - } - - /// Specify the index of files in the stream which should - /// throw an error when scanning - pub fn with_scan_errors(mut self, idx: Vec) -> Self { - self.opener.error_scanning_idx = idx; - self - } - - /// Specify the behavior of the stream when an error occurs - pub fn with_on_error(mut self, on_error: OnError) -> Self { - self.on_error = on_error; - self - } - - /// Specify the record batches that should be returned from each - /// file that is successfully scanned - pub fn with_records(mut self, records: Vec) -> Self { - self.opener.records = records; - self - } - - /// Collect the results of the `FileStream` - pub async fn result(self) -> Result> { - let file_schema = self - .opener - .records - .first() - .map(|batch| batch.schema()) - .unwrap_or_else(|| Arc::new(Schema::empty())); - - let ctx = SessionContext::new(); - let mock_files: Vec<(String, u64)> = (0..self.num_files) - .map(|idx| (format!("mock_file{idx}"), 10_u64)) - .collect(); - - let mock_files_ref: Vec<(&str, u64)> = mock_files - .iter() - .map(|(name, size)| (name.as_str(), *size)) - .collect(); - - register_test_store(&ctx, &mock_files_ref); - - let file_group = mock_files - .into_iter() - .map(|(name, size)| PartitionedFile::new(name, size)) - .collect(); - - let on_error = self.on_error; - - let config = FileScanConfig::new( - ObjectStoreUrl::parse("test:///").unwrap(), - file_schema, - Arc::new(CsvSource::default()), - ) - .with_file_group(file_group) - .with_limit(self.limit); - let metrics_set = ExecutionPlanMetricsSet::new(); - let file_stream = - FileStream::new(&config, 0, Arc::new(self.opener), &metrics_set) - .unwrap() - .with_on_error(on_error); - - file_stream - .collect::>() - .await - .into_iter() - .collect::>>() - } - } - - /// helper that creates a stream of 2 files with the same pair of batches in each ([0,1,2] and [0,1]) - async fn create_and_collect(limit: Option) -> Vec { - FileStreamTest::new() - .with_records(vec![make_partition(3), make_partition(2)]) - .with_num_files(2) - .with_limit(limit) - .result() - .await - .expect("error executing stream") - } - - #[tokio::test] - async fn on_error_opening() -> Result<()> { - let batches = FileStreamTest::new() - .with_records(vec![make_partition(3), make_partition(2)]) - .with_num_files(2) - .with_on_error(OnError::Skip) - .with_open_errors(vec![0]) - .result() - .await?; - - #[rustfmt::skip] - crate::assert_batches_eq!(&[ - "+---+", - "| i |", - "+---+", - "| 0 |", - "| 1 |", - "| 2 |", - "| 0 |", - "| 1 |", - "+---+", - ], &batches); - - let batches = FileStreamTest::new() - .with_records(vec![make_partition(3), make_partition(2)]) - .with_num_files(2) - .with_on_error(OnError::Skip) - .with_open_errors(vec![1]) - .result() - .await?; - - #[rustfmt::skip] - crate::assert_batches_eq!(&[ - "+---+", - "| i |", - "+---+", - "| 0 |", - "| 1 |", - "| 2 |", - "| 0 |", - "| 1 |", - "+---+", - ], &batches); - - let batches = FileStreamTest::new() - .with_records(vec![make_partition(3), make_partition(2)]) - .with_num_files(2) - .with_on_error(OnError::Skip) - .with_open_errors(vec![0, 1]) - .result() - .await?; - - #[rustfmt::skip] - crate::assert_batches_eq!(&[ - "++", - "++", - ], &batches); - - Ok(()) - } - - #[tokio::test] - async fn on_error_scanning_fail() -> Result<()> { - let result = FileStreamTest::new() - .with_records(vec![make_partition(3), make_partition(2)]) - .with_num_files(2) - .with_on_error(OnError::Fail) - .with_scan_errors(vec![1]) - .result() - .await; - - assert!(result.is_err()); - - Ok(()) - } - - #[tokio::test] - async fn on_error_opening_fail() -> Result<()> { - let result = FileStreamTest::new() - .with_records(vec![make_partition(3), make_partition(2)]) - .with_num_files(2) - .with_on_error(OnError::Fail) - .with_open_errors(vec![1]) - .result() - .await; - - assert!(result.is_err()); - - Ok(()) - } - - #[tokio::test] - async fn on_error_scanning() -> Result<()> { - let batches = FileStreamTest::new() - .with_records(vec![make_partition(3), make_partition(2)]) - .with_num_files(2) - .with_on_error(OnError::Skip) - .with_scan_errors(vec![0]) - .result() - .await?; - - #[rustfmt::skip] - crate::assert_batches_eq!(&[ - "+---+", - "| i |", - "+---+", - "| 0 |", - "| 1 |", - "| 2 |", - "| 0 |", - "| 1 |", - "+---+", - ], &batches); - - let batches = FileStreamTest::new() - .with_records(vec![make_partition(3), make_partition(2)]) - .with_num_files(2) - .with_on_error(OnError::Skip) - .with_scan_errors(vec![1]) - .result() - .await?; - - #[rustfmt::skip] - crate::assert_batches_eq!(&[ - "+---+", - "| i |", - "+---+", - "| 0 |", - "| 1 |", - "| 2 |", - "| 0 |", - "| 1 |", - "+---+", - ], &batches); - - let batches = FileStreamTest::new() - .with_records(vec![make_partition(3), make_partition(2)]) - .with_num_files(2) - .with_on_error(OnError::Skip) - .with_scan_errors(vec![0, 1]) - .result() - .await?; - - #[rustfmt::skip] - crate::assert_batches_eq!(&[ - "++", - "++", - ], &batches); - - Ok(()) - } - - #[tokio::test] - async fn on_error_mixed() -> Result<()> { - let batches = FileStreamTest::new() - .with_records(vec![make_partition(3), make_partition(2)]) - .with_num_files(3) - .with_on_error(OnError::Skip) - .with_open_errors(vec![1]) - .with_scan_errors(vec![0]) - .result() - .await?; - - #[rustfmt::skip] - crate::assert_batches_eq!(&[ - "+---+", - "| i |", - "+---+", - "| 0 |", - "| 1 |", - "| 2 |", - "| 0 |", - "| 1 |", - "+---+", - ], &batches); - - let batches = FileStreamTest::new() - .with_records(vec![make_partition(3), make_partition(2)]) - .with_num_files(3) - .with_on_error(OnError::Skip) - .with_open_errors(vec![0]) - .with_scan_errors(vec![1]) - .result() - .await?; - - #[rustfmt::skip] - crate::assert_batches_eq!(&[ - "+---+", - "| i |", - "+---+", - "| 0 |", - "| 1 |", - "| 2 |", - "| 0 |", - "| 1 |", - "+---+", - ], &batches); - - let batches = FileStreamTest::new() - .with_records(vec![make_partition(3), make_partition(2)]) - .with_num_files(3) - .with_on_error(OnError::Skip) - .with_open_errors(vec![2]) - .with_scan_errors(vec![0, 1]) - .result() - .await?; - - #[rustfmt::skip] - crate::assert_batches_eq!(&[ - "++", - "++", - ], &batches); - - let batches = FileStreamTest::new() - .with_records(vec![make_partition(3), make_partition(2)]) - .with_num_files(3) - .with_on_error(OnError::Skip) - .with_open_errors(vec![0, 2]) - .with_scan_errors(vec![1]) - .result() - .await?; - - #[rustfmt::skip] - crate::assert_batches_eq!(&[ - "++", - "++", - ], &batches); - - Ok(()) - } - - #[tokio::test] - async fn without_limit() -> Result<()> { - let batches = create_and_collect(None).await; - - #[rustfmt::skip] - crate::assert_batches_eq!(&[ - "+---+", - "| i |", - "+---+", - "| 0 |", - "| 1 |", - "| 2 |", - "| 0 |", - "| 1 |", - "| 0 |", - "| 1 |", - "| 2 |", - "| 0 |", - "| 1 |", - "+---+", - ], &batches); - - Ok(()) - } - - #[tokio::test] - async fn with_limit_between_files() -> Result<()> { - let batches = create_and_collect(Some(5)).await; - #[rustfmt::skip] - crate::assert_batches_eq!(&[ - "+---+", - "| i |", - "+---+", - "| 0 |", - "| 1 |", - "| 2 |", - "| 0 |", - "| 1 |", - "+---+", - ], &batches); - - Ok(()) - } - - #[tokio::test] - async fn with_limit_at_middle_of_batch() -> Result<()> { - let batches = create_and_collect(Some(6)).await; - #[rustfmt::skip] - crate::assert_batches_eq!(&[ - "+---+", - "| i |", - "+---+", - "| 0 |", - "| 1 |", - "| 2 |", - "| 0 |", - "| 1 |", - "| 0 |", - "+---+", - ], &batches); - - Ok(()) - } -} diff --git a/datafusion/core/src/datasource/physical_plan/json.rs b/datafusion/core/src/datasource/physical_plan/json.rs index d1ae13b083ab..c92d4dfdf835 100644 --- a/datafusion/core/src/datasource/physical_plan/json.rs +++ b/datafusion/core/src/datasource/physical_plan/json.rs @@ -22,8 +22,8 @@ use std::io::{BufReader, Read, Seek, SeekFrom}; use std::sync::Arc; use std::task::Poll; -use super::{calculate_range, FileScanConfig, RangeCalculation}; -use crate::datasource::data_source::FileSource; +use super::{calculate_range, RangeCalculation}; + use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::file_format::{deserialize_stream, DecoderDeserializer}; use crate::datasource::listing::{ListingTableUrl, PartitionedFile}; @@ -35,6 +35,8 @@ use crate::physical_plan::{ExecutionPlan, ExecutionPlanProperties}; use arrow::json::ReaderBuilder; use arrow::{datatypes::SchemaRef, json}; use datafusion_common::{Constraints, Statistics}; +use datafusion_datasource::file::FileSource; +use datafusion_datasource::file_scan_config::FileScanConfig; use datafusion_datasource::source::DataSourceExec; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_physical_expr::{EquivalenceProperties, Partitioning}; diff --git a/datafusion/core/src/datasource/physical_plan/mod.rs b/datafusion/core/src/datasource/physical_plan/mod.rs index 953c99322e16..42f6912afec0 100644 --- a/datafusion/core/src/datasource/physical_plan/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/mod.rs @@ -20,12 +20,10 @@ mod arrow_file; mod avro; mod csv; -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; #[cfg(feature = "parquet")] @@ -35,12 +33,9 @@ pub use self::parquet::source::ParquetSource; pub use self::parquet::{ ParquetExec, ParquetExecBuilder, ParquetFileMetrics, ParquetFileReaderFactory, }; +use crate::datasource::listing::FileRange; use crate::error::Result; -use crate::physical_plan::{DisplayAs, DisplayFormatType}; -use crate::{ - datasource::listing::{FileRange, PartitionedFile}, - physical_plan::display::{display_orderings, ProjectSchemaDisplay}, -}; +use crate::physical_plan::DisplayAs; #[allow(deprecated)] pub use arrow_file::ArrowExec; pub use arrow_file::ArrowSource; @@ -50,300 +45,24 @@ pub use avro::AvroSource; #[allow(deprecated)] pub use csv::{CsvExec, CsvExecBuilder}; pub use csv::{CsvOpener, CsvSource}; +pub use datafusion_datasource::file::FileSource; pub use datafusion_datasource::file_groups::FileGroupPartitioner; pub use datafusion_datasource::file_meta::FileMeta; -pub use datafusion_datasource::file_sink_config::*; -pub use file_scan_config::{ +pub use datafusion_datasource::file_scan_config::{ wrap_partition_type_in_dict, wrap_partition_value_in_dict, FileScanConfig, }; -pub use file_stream::{FileOpenFuture, FileOpener, FileStream, OnError}; +pub use datafusion_datasource::file_sink_config::*; + +pub use datafusion_datasource::file_stream::{ + FileOpenFuture, FileOpener, FileStream, OnError, +}; use futures::StreamExt; #[allow(deprecated)] pub use json::NdJsonExec; pub use json::{JsonOpener, JsonSource}; -use log::debug; -use object_store::{path::Path, GetOptions, GetRange, ObjectStore}; -use std::{ - fmt::{Debug, Formatter, Result as FmtResult}, - ops::Range, - sync::Arc, - vec, -}; - -use arrow::datatypes::SchemaRef; -use datafusion_physical_expr::expressions::Column; -use datafusion_physical_expr::PhysicalSortExpr; -use datafusion_physical_expr_common::sort_expr::LexOrdering; - -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(()) -} - -/// 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 ┃ -/// ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ -/// -/// DataSourceExec -///``` -/// -/// 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 -/// ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ┛ -/// -/// DataSourceExec -///``` -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 -} +use object_store::{path::Path, GetOptions, GetRange, ObjectStore}; +use std::{ops::Range, sync::Arc}; /// Represents the possible outcomes of a range calculation. /// @@ -452,7 +171,6 @@ async fn find_first_newline( #[cfg(test)] mod tests { use super::*; - use crate::physical_plan::{DefaultDisplay, VerboseDisplay}; use arrow::array::{ cast::AsArray, @@ -461,12 +179,11 @@ mod tests { StringArray, UInt64Array, }; use arrow::datatypes::{DataType, Field, Schema}; - use object_store::ObjectMeta; + use arrow_schema::SchemaRef; use crate::datasource::schema_adapter::{ DefaultSchemaAdapterFactory, SchemaAdapterFactory, }; - use chrono::Utc; #[test] fn schema_mapping_map_batch() { @@ -584,157 +301,4 @@ mod tests { assert_eq!(c4.value(1), 2.0_f32); assert_eq!(c4.value(2), 3.0_f32); } - - #[test] - fn file_groups_display_empty() { - let expected = "{0 groups: []}"; - assert_eq!(DefaultDisplay(FileGroupsDisplay(&[])).to_string(), expected); - } - - #[test] - fn file_groups_display_one() { - let files = [vec![partitioned_file("foo"), partitioned_file("bar")]]; - - let expected = "{1 group: [[foo, bar]]}"; - assert_eq!( - DefaultDisplay(FileGroupsDisplay(&files)).to_string(), - expected - ); - } - - #[test] - fn file_groups_display_many_default() { - let files = [ - vec![partitioned_file("foo"), partitioned_file("bar")], - vec![partitioned_file("baz")], - vec![], - ]; - - let expected = "{3 groups: [[foo, bar], [baz], []]}"; - assert_eq!( - DefaultDisplay(FileGroupsDisplay(&files)).to_string(), - expected - ); - } - - #[test] - fn file_groups_display_many_verbose() { - let files = [ - vec![partitioned_file("foo"), partitioned_file("bar")], - vec![partitioned_file("baz")], - vec![], - ]; - - let expected = "{3 groups: [[foo, bar], [baz], []]}"; - assert_eq!( - VerboseDisplay(FileGroupsDisplay(&files)).to_string(), - expected - ); - } - - #[test] - fn file_groups_display_too_many_default() { - let files = [ - vec![partitioned_file("foo"), partitioned_file("bar")], - vec![partitioned_file("baz")], - vec![partitioned_file("qux")], - vec![partitioned_file("quux")], - vec![partitioned_file("quuux")], - vec![partitioned_file("quuuux")], - vec![], - ]; - - let expected = "{7 groups: [[foo, bar], [baz], [qux], [quux], [quuux], ...]}"; - assert_eq!( - DefaultDisplay(FileGroupsDisplay(&files)).to_string(), - expected - ); - } - - #[test] - fn file_groups_display_too_many_verbose() { - let files = [ - vec![partitioned_file("foo"), partitioned_file("bar")], - vec![partitioned_file("baz")], - vec![partitioned_file("qux")], - vec![partitioned_file("quux")], - vec![partitioned_file("quuux")], - vec![partitioned_file("quuuux")], - vec![], - ]; - - let expected = - "{7 groups: [[foo, bar], [baz], [qux], [quux], [quuux], [quuuux], []]}"; - assert_eq!( - VerboseDisplay(FileGroupsDisplay(&files)).to_string(), - expected - ); - } - - #[test] - fn file_group_display_many_default() { - let files = vec![partitioned_file("foo"), partitioned_file("bar")]; - - let expected = "[foo, bar]"; - assert_eq!( - DefaultDisplay(FileGroupDisplay(&files)).to_string(), - expected - ); - } - - #[test] - fn file_group_display_too_many_default() { - let files = vec![ - partitioned_file("foo"), - partitioned_file("bar"), - partitioned_file("baz"), - partitioned_file("qux"), - partitioned_file("quux"), - partitioned_file("quuux"), - ]; - - let expected = "[foo, bar, baz, qux, quux, ...]"; - assert_eq!( - DefaultDisplay(FileGroupDisplay(&files)).to_string(), - expected - ); - } - - #[test] - fn file_group_display_too_many_verbose() { - let files = vec![ - partitioned_file("foo"), - partitioned_file("bar"), - partitioned_file("baz"), - partitioned_file("qux"), - partitioned_file("quux"), - partitioned_file("quuux"), - ]; - - let expected = "[foo, bar, baz, qux, quux, quuux]"; - assert_eq!( - VerboseDisplay(FileGroupDisplay(&files)).to_string(), - expected - ); - } - - /// create a PartitionedFile for testing - fn partitioned_file(path: &str) -> PartitionedFile { - let object_meta = ObjectMeta { - location: Path::parse(path).unwrap(), - last_modified: Utc::now(), - size: 42, - e_tag: None, - version: None, - }; - - PartitionedFile { - object_meta, - partition_values: vec![], - range: None, - statistics: None, - extensions: None, - metadata_size_hint: None, - } - } } diff --git a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs index 2a2d6d7fefdf..89902336ce5e 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs @@ -32,9 +32,7 @@ use std::fmt::Formatter; use std::sync::Arc; use crate::datasource::listing::PartitionedFile; -use crate::datasource::physical_plan::{ - parquet::source::ParquetSource, DisplayAs, FileScanConfig, -}; +use crate::datasource::physical_plan::{parquet::source::ParquetSource, DisplayAs}; use crate::datasource::schema_adapter::SchemaAdapterFactory; use crate::{ config::TableParquetOptions, @@ -50,6 +48,7 @@ pub use access_plan::{ParquetAccessPlan, RowGroupAccess}; use arrow::datatypes::SchemaRef; use datafusion_common::config::ConfigOptions; use datafusion_common::Constraints; +use datafusion_datasource::file_scan_config::FileScanConfig; use datafusion_datasource::source::DataSourceExec; use datafusion_physical_expr::{EquivalenceProperties, LexOrdering, PhysicalExpr}; use datafusion_physical_optimizer::pruning::PruningPredicate; diff --git a/datafusion/core/src/datasource/physical_plan/parquet/source.rs b/datafusion/core/src/datasource/physical_plan/parquet/source.rs index 178de8f51ae4..0f0863905a3c 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/source.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/source.rs @@ -20,13 +20,10 @@ use std::any::Any; use std::fmt::Formatter; use std::sync::Arc; -use crate::datasource::data_source::FileSource; use crate::datasource::physical_plan::parquet::opener::ParquetOpener; use crate::datasource::physical_plan::parquet::page_filter::PagePruningAccessPlanFilter; use crate::datasource::physical_plan::parquet::DefaultParquetFileReaderFactory; -use crate::datasource::physical_plan::{ - FileOpener, FileScanConfig, ParquetFileReaderFactory, -}; +use crate::datasource::physical_plan::{FileOpener, ParquetFileReaderFactory}; use crate::datasource::schema_adapter::{ DefaultSchemaAdapterFactory, SchemaAdapterFactory, }; @@ -34,6 +31,8 @@ use crate::datasource::schema_adapter::{ use arrow::datatypes::{Schema, SchemaRef}; use datafusion_common::config::TableParquetOptions; use datafusion_common::Statistics; +use datafusion_datasource::file::FileSource; +use datafusion_datasource::file_scan_config::FileScanConfig; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; use datafusion_physical_optimizer::pruning::PruningPredicate; use datafusion_physical_plan::metrics::{ExecutionPlanMetricsSet, MetricBuilder}; diff --git a/datafusion/core/src/datasource/schema_adapter.rs b/datafusion/core/src/datasource/schema_adapter.rs index 41e375cf81f8..8076c114ad16 100644 --- a/datafusion/core/src/datasource/schema_adapter.rs +++ b/datafusion/core/src/datasource/schema_adapter.rs @@ -437,12 +437,13 @@ mod tests { use arrow::array::{Int32Array, StringArray}; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; + use datafusion_datasource::file_scan_config::FileScanConfig; use object_store::path::Path; use object_store::ObjectMeta; use crate::datasource::listing::PartitionedFile; use crate::datasource::object_store::ObjectStoreUrl; - use crate::datasource::physical_plan::{FileScanConfig, ParquetSource}; + use crate::datasource::physical_plan::ParquetSource; use crate::datasource::schema_adapter::{ DefaultSchemaAdapterFactory, SchemaAdapter, SchemaAdapterFactory, SchemaMapper, }; diff --git a/datafusion/core/src/test/mod.rs b/datafusion/core/src/test/mod.rs index c569113a27bd..489e1ed240f4 100644 --- a/datafusion/core/src/test/mod.rs +++ b/datafusion/core/src/test/mod.rs @@ -25,13 +25,12 @@ use std::io::{BufReader, BufWriter}; use std::path::Path; use std::sync::Arc; -use crate::datasource::data_source::FileSource; use crate::datasource::file_format::csv::CsvFormat; use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::file_format::FileFormat; use crate::datasource::listing::PartitionedFile; use crate::datasource::object_store::ObjectStoreUrl; -use crate::datasource::physical_plan::{CsvSource, FileScanConfig}; +use crate::datasource::physical_plan::CsvSource; use crate::datasource::{MemTable, TableProvider}; use crate::error::Result; use crate::logical_expr::LogicalPlan; @@ -42,6 +41,8 @@ use arrow::array::{self, Array, ArrayRef, Decimal128Builder, Int32Array}; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use datafusion_common::DataFusionError; +use datafusion_datasource::file::FileSource; +use datafusion_datasource::file_scan_config::FileScanConfig; use datafusion_datasource::source::DataSourceExec; #[cfg(feature = "compression")] diff --git a/datafusion/core/src/test_util/parquet.rs b/datafusion/core/src/test_util/parquet.rs index fc98b43051f8..9c6888bb8b10 100644 --- a/datafusion/core/src/test_util/parquet.rs +++ b/datafusion/core/src/test_util/parquet.rs @@ -26,7 +26,7 @@ use crate::common::ToDFSchema; use crate::config::ConfigOptions; use crate::datasource::listing::{ListingTableUrl, PartitionedFile}; use crate::datasource::object_store::ObjectStoreUrl; -use crate::datasource::physical_plan::{FileScanConfig, ParquetSource}; +use crate::datasource::physical_plan::ParquetSource; use crate::error::Result; use crate::logical_expr::execution_props::ExecutionProps; use crate::logical_expr::simplify::SimplifyContext; @@ -37,6 +37,7 @@ use crate::physical_plan::metrics::MetricsSet; use crate::physical_plan::ExecutionPlan; use crate::prelude::{Expr, SessionConfig, SessionContext}; +use datafusion_datasource::file_scan_config::FileScanConfig; use datafusion_datasource::source::DataSourceExec; use object_store::path::Path; use object_store::ObjectMeta; diff --git a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs index 3412b962d859..ae69b0609a5d 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs @@ -28,6 +28,7 @@ use crate::physical_optimizer::test_utils::{ spr_repartition_exec, stream_exec_ordered, union_exec, RequirementsTestExec, }; +use datafusion_datasource::file_scan_config::FileScanConfig; use datafusion_physical_plan::displayable; use arrow::compute::SortOptions; use arrow::datatypes::SchemaRef; @@ -46,7 +47,7 @@ use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeE use datafusion_physical_plan::{get_plan_string, ExecutionPlan}; use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{TreeNode, TransformedResult}; -use datafusion::datasource::physical_plan::{CsvSource, FileScanConfig, ParquetSource}; +use datafusion::datasource::physical_plan::{CsvSource, ParquetSource}; use datafusion_execution::object_store::ObjectStoreUrl; use datafusion::datasource::listing::PartitionedFile; use datafusion_physical_optimizer::enforce_distribution::EnforceDistribution; diff --git a/datafusion/core/tests/physical_optimizer/test_utils.rs b/datafusion/core/tests/physical_optimizer/test_utils.rs index e4d72c112c38..418c46628daa 100644 --- a/datafusion/core/tests/physical_optimizer/test_utils.rs +++ b/datafusion/core/tests/physical_optimizer/test_utils.rs @@ -27,12 +27,13 @@ use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use datafusion::datasource::listing::PartitionedFile; use datafusion::datasource::memory::MemorySourceConfig; -use datafusion::datasource::physical_plan::{FileScanConfig, ParquetSource}; +use datafusion::datasource::physical_plan::ParquetSource; use datafusion::datasource::source::DataSourceExec; use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::utils::expr::COUNT_STAR_EXPANSION; use datafusion_common::{JoinType, Result}; +use datafusion_datasource::file_scan_config::FileScanConfig; use datafusion_execution::object_store::ObjectStoreUrl; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_expr::{WindowFrame, WindowFunctionDefinition}; diff --git a/datafusion/datasource/Cargo.toml b/datafusion/datasource/Cargo.toml index a1d7af106f69..d75b27c2f685 100644 --- a/datafusion/datasource/Cargo.toml +++ b/datafusion/datasource/Cargo.toml @@ -31,6 +31,8 @@ version.workspace = true all-features = true [features] +# Temporary feature while I move things around +avro = [] compression = ["async-compression", "xz2", "bzip2", "flate2", "zstd", "tokio-util"] default = ["compression"] diff --git a/datafusion/datasource/src/display.rs b/datafusion/datasource/src/display.rs new file mode 100644 index 000000000000..58fc27bb8010 --- /dev/null +++ b/datafusion/datasource/src/display.rs @@ -0,0 +1,295 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use datafusion_physical_plan::{DisplayAs, DisplayFormatType}; + +use std::fmt::{Debug, Formatter, Result as FmtResult}; + +use crate::PartitionedFile; + +/// A wrapper to customize partitioned file display +/// +/// Prints in the format: +/// ```text +/// {NUM_GROUPS groups: [[file1, file2,...], [fileN, fileM, ...], ...]} +/// ``` +#[derive(Debug)] +pub(crate) struct FileGroupsDisplay<'a>(pub(crate) &'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(()) +} + +#[cfg(test)] +mod tests { + use super::*; + + use datafusion_physical_plan::{DefaultDisplay, VerboseDisplay}; + use object_store::{path::Path, ObjectMeta}; + + use chrono::Utc; + + #[test] + fn file_groups_display_empty() { + let expected = "{0 groups: []}"; + assert_eq!(DefaultDisplay(FileGroupsDisplay(&[])).to_string(), expected); + } + + #[test] + fn file_groups_display_one() { + let files = [vec![partitioned_file("foo"), partitioned_file("bar")]]; + + let expected = "{1 group: [[foo, bar]]}"; + assert_eq!( + DefaultDisplay(FileGroupsDisplay(&files)).to_string(), + expected + ); + } + + #[test] + fn file_groups_display_many_default() { + let files = [ + vec![partitioned_file("foo"), partitioned_file("bar")], + vec![partitioned_file("baz")], + vec![], + ]; + + let expected = "{3 groups: [[foo, bar], [baz], []]}"; + assert_eq!( + DefaultDisplay(FileGroupsDisplay(&files)).to_string(), + expected + ); + } + + #[test] + fn file_groups_display_many_verbose() { + let files = [ + vec![partitioned_file("foo"), partitioned_file("bar")], + vec![partitioned_file("baz")], + vec![], + ]; + + let expected = "{3 groups: [[foo, bar], [baz], []]}"; + assert_eq!( + VerboseDisplay(FileGroupsDisplay(&files)).to_string(), + expected + ); + } + + #[test] + fn file_groups_display_too_many_default() { + let files = [ + vec![partitioned_file("foo"), partitioned_file("bar")], + vec![partitioned_file("baz")], + vec![partitioned_file("qux")], + vec![partitioned_file("quux")], + vec![partitioned_file("quuux")], + vec![partitioned_file("quuuux")], + vec![], + ]; + + let expected = "{7 groups: [[foo, bar], [baz], [qux], [quux], [quuux], ...]}"; + assert_eq!( + DefaultDisplay(FileGroupsDisplay(&files)).to_string(), + expected + ); + } + + #[test] + fn file_groups_display_too_many_verbose() { + let files = [ + vec![partitioned_file("foo"), partitioned_file("bar")], + vec![partitioned_file("baz")], + vec![partitioned_file("qux")], + vec![partitioned_file("quux")], + vec![partitioned_file("quuux")], + vec![partitioned_file("quuuux")], + vec![], + ]; + + let expected = + "{7 groups: [[foo, bar], [baz], [qux], [quux], [quuux], [quuuux], []]}"; + assert_eq!( + VerboseDisplay(FileGroupsDisplay(&files)).to_string(), + expected + ); + } + + #[test] + fn file_group_display_many_default() { + let files = vec![partitioned_file("foo"), partitioned_file("bar")]; + + let expected = "[foo, bar]"; + assert_eq!( + DefaultDisplay(FileGroupDisplay(&files)).to_string(), + expected + ); + } + + #[test] + fn file_group_display_too_many_default() { + let files = vec![ + partitioned_file("foo"), + partitioned_file("bar"), + partitioned_file("baz"), + partitioned_file("qux"), + partitioned_file("quux"), + partitioned_file("quuux"), + ]; + + let expected = "[foo, bar, baz, qux, quux, ...]"; + assert_eq!( + DefaultDisplay(FileGroupDisplay(&files)).to_string(), + expected + ); + } + + #[test] + fn file_group_display_too_many_verbose() { + let files = vec![ + partitioned_file("foo"), + partitioned_file("bar"), + partitioned_file("baz"), + partitioned_file("qux"), + partitioned_file("quux"), + partitioned_file("quuux"), + ]; + + let expected = "[foo, bar, baz, qux, quux, quuux]"; + assert_eq!( + VerboseDisplay(FileGroupDisplay(&files)).to_string(), + expected + ); + } + + /// create a PartitionedFile for testing + fn partitioned_file(path: &str) -> PartitionedFile { + let object_meta = ObjectMeta { + location: Path::parse(path).unwrap(), + last_modified: Utc::now(), + size: 42, + e_tag: None, + version: None, + }; + + PartitionedFile { + object_meta, + partition_values: vec![], + range: None, + statistics: None, + extensions: None, + metadata_size_hint: None, + } + } +} diff --git a/datafusion/core/src/datasource/data_source.rs b/datafusion/datasource/src/file.rs similarity index 95% rename from datafusion/core/src/datasource/data_source.rs rename to datafusion/datasource/src/file.rs index 2db79c5c839d..8d8cbbc67b9a 100644 --- a/datafusion/core/src/datasource/data_source.rs +++ b/datafusion/datasource/src/file.rs @@ -15,18 +15,18 @@ // specific language governing permissions and limitations // under the License. -//! DataSource and FileSource trait implementations +//! Common behaviors that every file format needs to implement use std::any::Any; use std::fmt; use std::fmt::Formatter; use std::sync::Arc; -use crate::datasource::physical_plan::{FileOpener, FileScanConfig}; - +use crate::file_groups::FileGroupPartitioner; +use crate::file_scan_config::FileScanConfig; +use crate::file_stream::FileOpener; use arrow::datatypes::SchemaRef; use datafusion_common::Statistics; -use datafusion_datasource::file_groups::FileGroupPartitioner; use datafusion_physical_expr::LexOrdering; use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; use datafusion_physical_plan::DisplayFormatType; diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index bfddbc3a1fc4..affea6e34afd 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -15,7 +15,13 @@ // specific language governing permissions and limitations // under the License. -use std::{borrow::Cow, collections::HashMap, marker::PhantomData, sync::Arc}; +//! [`FileScanConfig`] to configure scanning of possibly partitioned +//! file sources. + +use std::{ + any::Any, borrow::Cow, collections::HashMap, fmt::Debug, fmt::Formatter, + fmt::Result as FmtResult, marker::PhantomData, sync::Arc, +}; use arrow::{ array::{ @@ -23,11 +29,618 @@ use arrow::{ RecordBatchOptions, }, buffer::Buffer, - datatypes::{ArrowNativeType, DataType, SchemaRef, UInt16Type}, + datatypes::{ArrowNativeType, DataType, Field, Schema, SchemaRef, UInt16Type}, +}; +use datafusion_common::{ + exec_err, stats::Precision, ColumnStatistics, Constraints, Result, Statistics, }; -use datafusion_common::{exec_err, Result}; use datafusion_common::{DataFusionError, ScalarValue}; -use log::warn; +use datafusion_execution::{ + object_store::ObjectStoreUrl, SendableRecordBatchStream, TaskContext, +}; +use datafusion_physical_expr::{ + expressions::Column, EquivalenceProperties, LexOrdering, Partitioning, + PhysicalSortExpr, +}; +use datafusion_physical_plan::{ + display::{display_orderings, ProjectSchemaDisplay}, + metrics::ExecutionPlanMetricsSet, + projection::{all_alias_free_columns, new_projections_for_columns, ProjectionExec}, + DisplayAs, DisplayFormatType, ExecutionPlan, +}; +use log::{debug, warn}; + +use crate::{ + display::FileGroupsDisplay, + file::FileSource, + file_compression_type::FileCompressionType, + file_stream::FileStream, + source::{DataSource, DataSourceExec}, + statistics::MinMaxStatistics, + PartitionedFile, +}; + +/// The base configurations for a [`DataSourceExec`], the a physical plan for +/// any given file format. +/// +/// Use [`Self::build`] to create a [`DataSourceExec`] from a ``FileScanConfig`. +/// +/// # Example +/// ``` +/// # use std::any::Any; +/// # use std::sync::Arc; +/// # use arrow::datatypes::{Field, Fields, DataType, Schema, SchemaRef}; +/// # use object_store::ObjectStore; +/// # use datafusion_common::Statistics; +/// # use datafusion_datasource::file::FileSource; +/// # use datafusion_datasource::PartitionedFile; +/// # use datafusion_datasource::file_scan_config::FileScanConfig; +/// # use datafusion_datasource::file_stream::FileOpener; +/// # use datafusion_execution::object_store::ObjectStoreUrl; +/// # use datafusion_physical_plan::ExecutionPlan; +/// # use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; +/// # let file_schema = Arc::new(Schema::new(vec![ +/// # Field::new("c1", DataType::Int32, false), +/// # Field::new("c2", DataType::Int32, false), +/// # Field::new("c3", DataType::Int32, false), +/// # Field::new("c4", DataType::Int32, false), +/// # ])); +/// # // Note: crate mock ParquetSource, as ParquetSource is not in the datasource crate +/// # struct ParquetSource {}; +/// # impl FileSource for ParquetSource { +/// # fn create_file_opener(&self, _: Arc, _: &FileScanConfig, _: usize) -> Arc { unimplemented!() } +/// # fn as_any(&self) -> &dyn Any { self } +/// # fn with_batch_size(&self, _: usize) -> Arc { unimplemented!() } +/// # fn with_schema(&self, _: SchemaRef) -> Arc { unimplemented!() } +/// # fn with_projection(&self, _: &FileScanConfig) -> Arc { unimplemented!() } +/// # fn with_statistics(&self, _: Statistics) -> Arc { Arc::new(Self::new()) } +/// # fn metrics(&self) -> &ExecutionPlanMetricsSet { unimplemented!() } +/// # fn statistics(&self) -> datafusion_common::Result { unimplemented!() } +/// # fn file_type(&self) -> &str { "parquet" } +/// # } +/// # impl ParquetSource { +/// # fn new() -> Self { Self{} } +/// # } +/// // create FileScan config for reading parquet files from file:// +/// let object_store_url = ObjectStoreUrl::local_filesystem(); +/// let file_source = Arc::new(ParquetSource::new()); +/// let config = FileScanConfig::new(object_store_url, file_schema, file_source) +/// .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), +/// ]); +/// // create an execution plan from the config +/// let plan: Arc = config.build(); +/// ``` +#[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, + /// File compression type + pub file_compression_type: FileCompressionType, + /// Are new lines in values supported for CSVOptions + pub new_lines_in_values: bool, + /// File source such as `ParquetSource`, `CsvSource`, `JsonSource`, etc. + pub source: Arc, +} + +impl DataSource for FileScanConfig { + fn open( + &self, + partition: usize, + context: Arc, + ) -> Result { + let object_store = context.runtime_env().object_store(&self.object_store_url)?; + + let source = self + .source + .with_batch_size(context.session_config().batch_size()) + .with_schema(Arc::clone(&self.file_schema)) + .with_projection(self); + + let opener = source.create_file_opener(object_store, self, partition); + + let stream = FileStream::new(self, partition, opener, source.metrics())?; + Ok(Box::pin(stream)) + } + + fn as_any(&self) -> &dyn Any { + self + } + + 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)?; + } + + self.fmt_file_source(t, f) + } + + /// If supported by the underlying [`FileSource`], redistribute files across partitions according to their size. + fn repartitioned( + &self, + target_partitions: usize, + repartition_file_min_size: usize, + output_ordering: Option, + ) -> Result>> { + let source = self.source.repartitioned( + target_partitions, + repartition_file_min_size, + output_ordering, + self, + )?; + + Ok(source.map(|s| Arc::new(s) as _)) + } + + fn output_partitioning(&self) -> Partitioning { + Partitioning::UnknownPartitioning(self.file_groups.len()) + } + + fn eq_properties(&self) -> EquivalenceProperties { + let (schema, constraints, _, orderings) = self.project(); + EquivalenceProperties::new_with_orderings(schema, orderings.as_slice()) + .with_constraints(constraints) + } + + fn statistics(&self) -> Result { + self.source.statistics() + } + + fn with_fetch(&self, limit: Option) -> Option> { + let source = self.clone(); + Some(Arc::new(source.with_limit(limit))) + } + + fn fetch(&self) -> Option { + self.limit + } + + fn metrics(&self) -> ExecutionPlanMetricsSet { + self.source.metrics().clone() + } + + fn try_swapping_with_projection( + &self, + projection: &ProjectionExec, + ) -> Result>> { + // If there is any non-column or alias-carrier expression, Projection should not be removed. + // This process can be moved into CsvExec, but it would be an overlap of their responsibility. + Ok(all_alias_free_columns(projection.expr()).then(|| { + let file_scan = self.clone(); + let source = Arc::clone(&file_scan.source); + let new_projections = new_projections_for_columns( + projection, + &file_scan + .projection + .clone() + .unwrap_or((0..self.file_schema.fields().len()).collect()), + ); + file_scan + // Assign projected statistics to source + .with_projection(Some(new_projections)) + .with_source(source) + .build() as _ + })) + } +} + +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, + file_source: Arc, + ) -> Self { + let statistics = Statistics::new_unknown(&file_schema); + + let mut config = Self { + object_store_url, + file_schema, + file_groups: vec![], + constraints: Constraints::empty(), + statistics, + projection: None, + limit: None, + table_partition_cols: vec![], + output_ordering: vec![], + file_compression_type: FileCompressionType::UNCOMPRESSED, + new_lines_in_values: false, + source: Arc::clone(&file_source), + }; + + config = config.with_source(Arc::clone(&file_source)); + config + } + + /// Set the file source + pub fn with_source(mut self, source: Arc) -> Self { + let ( + _projected_schema, + _constraints, + projected_statistics, + _projected_output_ordering, + ) = self.project(); + self.source = source.with_statistics(projected_statistics); + self + } + + /// 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 + } + + /// Set the file compression type + pub fn with_file_compression_type( + mut self, + file_compression_type: FileCompressionType, + ) -> Self { + self.file_compression_type = file_compression_type; + self + } + + /// Set the new_lines_in_values property + pub fn with_newlines_in_values(mut self, new_lines_in_values: bool) -> Self { + self.new_lines_in_values = new_lines_in_values; + self + } + + /// Specifies whether newlines in (quoted) values are supported. + /// + /// Parsing newlines in quoted values may be affected by execution behaviour such as + /// parallel file scanning. Setting this to `true` ensures that newlines in values are + /// parsed successfully, which may reduce performance. + /// + /// The default behaviour depends on the `datafusion.catalog.newlines_in_values` setting. + pub fn newlines_in_values(&self) -> bool { + self.new_lines_in_values + } + + /// 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()) + } + + // TODO: This function should be moved into DataSourceExec once FileScanConfig moved out of datafusion/core + /// Returns a new [`DataSourceExec`] to scan the files specified by this config + pub fn build(self) -> Arc { + Arc::new(DataSourceExec::new(Arc::new(self))) + } + + /// Write the data_type based on file_source + fn fmt_file_source(&self, t: DisplayFormatType, f: &mut Formatter) -> FmtResult { + write!(f, ", file_type={}", self.source.file_type())?; + self.source.fmt_extra(t, f) + } + + /// Returns the file_source + pub fn file_source(&self) -> &Arc { + &self.source + } +} + +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. /// @@ -276,3 +889,829 @@ fn create_output_array( val.to_array_of_size(len) } + +/// 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 ┃ +/// ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ +/// +/// DataSourceExec +///``` +/// +/// 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 +/// ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ┛ +/// +/// DataSourceExec +///``` +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. +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)) +} + +#[cfg(test)] +mod tests { + use crate::{test_util::MockSource, tests::aggr_test_schema}; + + use super::*; + use arrow::{ + array::{Int32Array, RecordBatch}, + compute::SortOptions, + }; + + use datafusion_common::{assert_batches_eq, DFSchema}; + use datafusion_expr::{execution_props::ExecutionProps, SortExpr}; + use datafusion_physical_expr::create_physical_expr; + use std::collections::HashMap; + + 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, + }, + }) + } + + /// Returns the column names on the schema + pub 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::>(), + )); + 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, + Arc::new(MockSource::default()), + ) + .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/datasource/src/file_stream.rs b/datafusion/datasource/src/file_stream.rs index 570ca6678538..7d17d230fc01 100644 --- a/datafusion/datasource/src/file_stream.rs +++ b/datafusion/datasource/src/file_stream.rs @@ -21,10 +21,20 @@ //! Note: Most traits here need to be marked `Sync + Send` to be //! compliant with the `SendableRecordBatchStream` trait. +use std::collections::VecDeque; +use std::mem; +use std::pin::Pin; +use std::sync::Arc; +use std::task::{Context, Poll}; + use crate::file_meta::FileMeta; +use crate::file_scan_config::{FileScanConfig, PartitionColumnProjector}; +use crate::PartitionedFile; +use arrow::datatypes::SchemaRef; use datafusion_common::error::Result; +use datafusion_execution::RecordBatchStream; use datafusion_physical_plan::metrics::{ - Count, ExecutionPlanMetricsSet, MetricBuilder, Time, + BaselineMetrics, Count, ExecutionPlanMetricsSet, MetricBuilder, Time, }; use arrow::error::ArrowError; @@ -34,6 +44,303 @@ use datafusion_common::ScalarValue; use futures::future::BoxFuture; use futures::stream::BoxStream; +use futures::{ready, FutureExt as _, Stream, StreamExt as _}; + +/// A stream that iterates record batch by record batch, file over file. +pub struct FileStream { + /// An iterator over input files. + file_iter: VecDeque, + /// The stream schema (file schema including partition columns and after + /// projection). + projected_schema: SchemaRef, + /// The remaining number of records to parse, None if no limit + remain: Option, + /// A dynamic [`FileOpener`]. Calling `open()` returns a [`FileOpenFuture`], + /// which can be resolved to a stream of `RecordBatch`. + file_opener: Arc, + /// The partition column projector + pc_projector: PartitionColumnProjector, + /// The stream state + state: FileStreamState, + /// File stream specific metrics + file_stream_metrics: FileStreamMetrics, + /// runtime baseline metrics + baseline_metrics: BaselineMetrics, + /// Describes the behavior of the `FileStream` if file opening or scanning fails + on_error: OnError, +} + +impl FileStream { + /// Create a new `FileStream` using the give `FileOpener` to scan underlying files + pub fn new( + config: &FileScanConfig, + partition: usize, + file_opener: Arc, + metrics: &ExecutionPlanMetricsSet, + ) -> Result { + let (projected_schema, ..) = config.project(); + let pc_projector = PartitionColumnProjector::new( + Arc::clone(&projected_schema), + &config + .table_partition_cols + .iter() + .map(|x| x.name().clone()) + .collect::>(), + ); + + let files = config.file_groups[partition].clone(); + + Ok(Self { + file_iter: files.into(), + projected_schema, + remain: config.limit, + file_opener, + pc_projector, + state: FileStreamState::Idle, + file_stream_metrics: FileStreamMetrics::new(metrics, partition), + baseline_metrics: BaselineMetrics::new(metrics, partition), + on_error: OnError::Fail, + }) + } + + /// Specify the behavior when an error occurs opening or scanning a file + /// + /// If `OnError::Skip` the stream will skip files which encounter an error and continue + /// If `OnError:Fail` (default) the stream will fail and stop processing when an error occurs + pub fn with_on_error(mut self, on_error: OnError) -> Self { + self.on_error = on_error; + self + } + + /// Begin opening the next file in parallel while decoding the current file in FileStream. + /// + /// Since file opening is mostly IO (and may involve a + /// bunch of sequential IO), it can be parallelized with decoding. + fn start_next_file(&mut self) -> Option)>> { + let part_file = self.file_iter.pop_front()?; + + let file_meta = FileMeta { + object_meta: part_file.object_meta, + range: part_file.range, + extensions: part_file.extensions, + metadata_size_hint: part_file.metadata_size_hint, + }; + + Some( + self.file_opener + .open(file_meta) + .map(|future| (future, part_file.partition_values)), + ) + } + + fn poll_inner(&mut self, cx: &mut Context<'_>) -> Poll>> { + loop { + match &mut self.state { + FileStreamState::Idle => { + self.file_stream_metrics.time_opening.start(); + + match self.start_next_file().transpose() { + Ok(Some((future, partition_values))) => { + self.state = FileStreamState::Open { + future, + partition_values, + } + } + Ok(None) => return Poll::Ready(None), + Err(e) => { + self.state = FileStreamState::Error; + return Poll::Ready(Some(Err(e))); + } + } + } + FileStreamState::Open { + future, + partition_values, + } => match ready!(future.poll_unpin(cx)) { + Ok(reader) => { + let partition_values = mem::take(partition_values); + + // include time needed to start opening in `start_next_file` + self.file_stream_metrics.time_opening.stop(); + let next = self.start_next_file().transpose(); + self.file_stream_metrics.time_scanning_until_data.start(); + self.file_stream_metrics.time_scanning_total.start(); + + match next { + Ok(Some((next_future, next_partition_values))) => { + self.state = FileStreamState::Scan { + partition_values, + reader, + next: Some(( + NextOpen::Pending(next_future), + next_partition_values, + )), + }; + } + Ok(None) => { + self.state = FileStreamState::Scan { + reader, + partition_values, + next: None, + }; + } + Err(e) => { + self.state = FileStreamState::Error; + return Poll::Ready(Some(Err(e))); + } + } + } + Err(e) => { + self.file_stream_metrics.file_open_errors.add(1); + match self.on_error { + OnError::Skip => { + self.file_stream_metrics.time_opening.stop(); + self.state = FileStreamState::Idle + } + OnError::Fail => { + self.state = FileStreamState::Error; + return Poll::Ready(Some(Err(e))); + } + } + } + }, + FileStreamState::Scan { + reader, + partition_values, + next, + } => { + // We need to poll the next `FileOpenFuture` here to drive it forward + if let Some((next_open_future, _)) = next { + if let NextOpen::Pending(f) = next_open_future { + if let Poll::Ready(reader) = f.as_mut().poll(cx) { + *next_open_future = NextOpen::Ready(reader); + } + } + } + match ready!(reader.poll_next_unpin(cx)) { + Some(Ok(batch)) => { + self.file_stream_metrics.time_scanning_until_data.stop(); + self.file_stream_metrics.time_scanning_total.stop(); + let result = self + .pc_projector + .project(batch, partition_values) + .map_err(|e| ArrowError::ExternalError(e.into())) + .map(|batch| match &mut self.remain { + Some(remain) => { + if *remain > batch.num_rows() { + *remain -= batch.num_rows(); + batch + } else { + let batch = batch.slice(0, *remain); + self.state = FileStreamState::Limit; + *remain = 0; + batch + } + } + None => batch, + }); + + if result.is_err() { + // If the partition value projection fails, this is not governed by + // the `OnError` behavior + self.state = FileStreamState::Error + } + self.file_stream_metrics.time_scanning_total.start(); + return Poll::Ready(Some(result.map_err(Into::into))); + } + Some(Err(err)) => { + self.file_stream_metrics.file_scan_errors.add(1); + self.file_stream_metrics.time_scanning_until_data.stop(); + self.file_stream_metrics.time_scanning_total.stop(); + + match self.on_error { + // If `OnError::Skip` we skip the file as soon as we hit the first error + OnError::Skip => match mem::take(next) { + Some((future, partition_values)) => { + self.file_stream_metrics.time_opening.start(); + + match future { + NextOpen::Pending(future) => { + self.state = FileStreamState::Open { + future, + partition_values, + } + } + NextOpen::Ready(reader) => { + self.state = FileStreamState::Open { + future: Box::pin(std::future::ready( + reader, + )), + partition_values, + } + } + } + } + None => return Poll::Ready(None), + }, + OnError::Fail => { + self.state = FileStreamState::Error; + return Poll::Ready(Some(Err(err.into()))); + } + } + } + None => { + self.file_stream_metrics.time_scanning_until_data.stop(); + self.file_stream_metrics.time_scanning_total.stop(); + + match mem::take(next) { + Some((future, partition_values)) => { + self.file_stream_metrics.time_opening.start(); + + match future { + NextOpen::Pending(future) => { + self.state = FileStreamState::Open { + future, + partition_values, + } + } + NextOpen::Ready(reader) => { + self.state = FileStreamState::Open { + future: Box::pin(std::future::ready( + reader, + )), + partition_values, + } + } + } + } + None => return Poll::Ready(None), + } + } + } + } + FileStreamState::Error | FileStreamState::Limit => { + return Poll::Ready(None) + } + } + } + } +} + +impl Stream for FileStream { + type Item = Result; + + fn poll_next( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll> { + self.file_stream_metrics.time_processing.start(); + let result = self.poll_inner(cx); + self.file_stream_metrics.time_processing.stop(); + self.baseline_metrics.record_poll(result) + } +} + +impl RecordBatchStream for FileStream { + fn schema(&self) -> SchemaRef { + Arc::clone(&self.projected_schema) + } +} /// A fallible future that resolves to a stream of [`RecordBatch`] pub type FileOpenFuture = @@ -212,3 +519,467 @@ impl FileStreamMetrics { } } } + +#[cfg(test)] +mod tests { + use crate::file_scan_config::FileScanConfig; + use crate::tests::make_partition; + use crate::PartitionedFile; + use arrow::error::ArrowError; + use datafusion_common::error::Result; + use datafusion_execution::object_store::ObjectStoreUrl; + use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; + use futures::{FutureExt as _, StreamExt as _}; + use std::sync::atomic::{AtomicUsize, Ordering}; + use std::sync::Arc; + + use crate::file_meta::FileMeta; + use crate::file_stream::{FileOpenFuture, FileOpener, FileStream, OnError}; + use crate::test_util::MockSource; + use arrow::array::RecordBatch; + use arrow::datatypes::Schema; + + use datafusion_common::{assert_batches_eq, internal_err}; + + /// Test `FileOpener` which will simulate errors during file opening or scanning + #[derive(Default)] + struct TestOpener { + /// Index in stream of files which should throw an error while opening + error_opening_idx: Vec, + /// Index in stream of files which should throw an error while scanning + error_scanning_idx: Vec, + /// Index of last file in stream + current_idx: AtomicUsize, + /// `RecordBatch` to return + records: Vec, + } + + impl FileOpener for TestOpener { + fn open(&self, _file_meta: FileMeta) -> Result { + let idx = self.current_idx.fetch_add(1, Ordering::SeqCst); + + if self.error_opening_idx.contains(&idx) { + Ok(futures::future::ready(internal_err!("error opening")).boxed()) + } else if self.error_scanning_idx.contains(&idx) { + let error = futures::future::ready(Err(ArrowError::IpcError( + "error scanning".to_owned(), + ))); + let stream = futures::stream::once(error).boxed(); + Ok(futures::future::ready(Ok(stream)).boxed()) + } else { + let iterator = self.records.clone().into_iter().map(Ok); + let stream = futures::stream::iter(iterator).boxed(); + Ok(futures::future::ready(Ok(stream)).boxed()) + } + } + } + + #[derive(Default)] + struct FileStreamTest { + /// Number of files in the stream + num_files: usize, + /// Global limit of records emitted by the stream + limit: Option, + /// Error-handling behavior of the stream + on_error: OnError, + /// Mock `FileOpener` + opener: TestOpener, + } + + impl FileStreamTest { + pub fn new() -> Self { + Self::default() + } + + /// Specify the number of files in the stream + pub fn with_num_files(mut self, num_files: usize) -> Self { + self.num_files = num_files; + self + } + + /// Specify the limit + pub fn with_limit(mut self, limit: Option) -> Self { + self.limit = limit; + self + } + + /// Specify the index of files in the stream which should + /// throw an error when opening + pub fn with_open_errors(mut self, idx: Vec) -> Self { + self.opener.error_opening_idx = idx; + self + } + + /// Specify the index of files in the stream which should + /// throw an error when scanning + pub fn with_scan_errors(mut self, idx: Vec) -> Self { + self.opener.error_scanning_idx = idx; + self + } + + /// Specify the behavior of the stream when an error occurs + pub fn with_on_error(mut self, on_error: OnError) -> Self { + self.on_error = on_error; + self + } + + /// Specify the record batches that should be returned from each + /// file that is successfully scanned + pub fn with_records(mut self, records: Vec) -> Self { + self.opener.records = records; + self + } + + /// Collect the results of the `FileStream` + pub async fn result(self) -> Result> { + let file_schema = self + .opener + .records + .first() + .map(|batch| batch.schema()) + .unwrap_or_else(|| Arc::new(Schema::empty())); + + // let ctx = SessionContext::new(); + let mock_files: Vec<(String, u64)> = (0..self.num_files) + .map(|idx| (format!("mock_file{idx}"), 10_u64)) + .collect(); + + // let mock_files_ref: Vec<(&str, u64)> = mock_files + // .iter() + // .map(|(name, size)| (name.as_str(), *size)) + // .collect(); + + let file_group = mock_files + .into_iter() + .map(|(name, size)| PartitionedFile::new(name, size)) + .collect(); + + let on_error = self.on_error; + + let config = FileScanConfig::new( + ObjectStoreUrl::parse("test:///").unwrap(), + file_schema, + Arc::new(MockSource::default()), + ) + .with_file_group(file_group) + .with_limit(self.limit); + let metrics_set = ExecutionPlanMetricsSet::new(); + let file_stream = + FileStream::new(&config, 0, Arc::new(self.opener), &metrics_set) + .unwrap() + .with_on_error(on_error); + + file_stream + .collect::>() + .await + .into_iter() + .collect::>>() + } + } + + /// helper that creates a stream of 2 files with the same pair of batches in each ([0,1,2] and [0,1]) + async fn create_and_collect(limit: Option) -> Vec { + FileStreamTest::new() + .with_records(vec![make_partition(3), make_partition(2)]) + .with_num_files(2) + .with_limit(limit) + .result() + .await + .expect("error executing stream") + } + + #[tokio::test] + async fn on_error_opening() -> Result<()> { + let batches = FileStreamTest::new() + .with_records(vec![make_partition(3), make_partition(2)]) + .with_num_files(2) + .with_on_error(OnError::Skip) + .with_open_errors(vec![0]) + .result() + .await?; + + #[rustfmt::skip] + assert_batches_eq!(&[ + "+---+", + "| i |", + "+---+", + "| 0 |", + "| 1 |", + "| 2 |", + "| 0 |", + "| 1 |", + "+---+", + ], &batches); + + let batches = FileStreamTest::new() + .with_records(vec![make_partition(3), make_partition(2)]) + .with_num_files(2) + .with_on_error(OnError::Skip) + .with_open_errors(vec![1]) + .result() + .await?; + + #[rustfmt::skip] + assert_batches_eq!(&[ + "+---+", + "| i |", + "+---+", + "| 0 |", + "| 1 |", + "| 2 |", + "| 0 |", + "| 1 |", + "+---+", + ], &batches); + + let batches = FileStreamTest::new() + .with_records(vec![make_partition(3), make_partition(2)]) + .with_num_files(2) + .with_on_error(OnError::Skip) + .with_open_errors(vec![0, 1]) + .result() + .await?; + + #[rustfmt::skip] + assert_batches_eq!(&[ + "++", + "++", + ], &batches); + + Ok(()) + } + + #[tokio::test] + async fn on_error_scanning_fail() -> Result<()> { + let result = FileStreamTest::new() + .with_records(vec![make_partition(3), make_partition(2)]) + .with_num_files(2) + .with_on_error(OnError::Fail) + .with_scan_errors(vec![1]) + .result() + .await; + + assert!(result.is_err()); + + Ok(()) + } + + #[tokio::test] + async fn on_error_opening_fail() -> Result<()> { + let result = FileStreamTest::new() + .with_records(vec![make_partition(3), make_partition(2)]) + .with_num_files(2) + .with_on_error(OnError::Fail) + .with_open_errors(vec![1]) + .result() + .await; + + assert!(result.is_err()); + + Ok(()) + } + + #[tokio::test] + async fn on_error_scanning() -> Result<()> { + let batches = FileStreamTest::new() + .with_records(vec![make_partition(3), make_partition(2)]) + .with_num_files(2) + .with_on_error(OnError::Skip) + .with_scan_errors(vec![0]) + .result() + .await?; + + #[rustfmt::skip] + assert_batches_eq!(&[ + "+---+", + "| i |", + "+---+", + "| 0 |", + "| 1 |", + "| 2 |", + "| 0 |", + "| 1 |", + "+---+", + ], &batches); + + let batches = FileStreamTest::new() + .with_records(vec![make_partition(3), make_partition(2)]) + .with_num_files(2) + .with_on_error(OnError::Skip) + .with_scan_errors(vec![1]) + .result() + .await?; + + #[rustfmt::skip] + assert_batches_eq!(&[ + "+---+", + "| i |", + "+---+", + "| 0 |", + "| 1 |", + "| 2 |", + "| 0 |", + "| 1 |", + "+---+", + ], &batches); + + let batches = FileStreamTest::new() + .with_records(vec![make_partition(3), make_partition(2)]) + .with_num_files(2) + .with_on_error(OnError::Skip) + .with_scan_errors(vec![0, 1]) + .result() + .await?; + + #[rustfmt::skip] + assert_batches_eq!(&[ + "++", + "++", + ], &batches); + + Ok(()) + } + + #[tokio::test] + async fn on_error_mixed() -> Result<()> { + let batches = FileStreamTest::new() + .with_records(vec![make_partition(3), make_partition(2)]) + .with_num_files(3) + .with_on_error(OnError::Skip) + .with_open_errors(vec![1]) + .with_scan_errors(vec![0]) + .result() + .await?; + + #[rustfmt::skip] + assert_batches_eq!(&[ + "+---+", + "| i |", + "+---+", + "| 0 |", + "| 1 |", + "| 2 |", + "| 0 |", + "| 1 |", + "+---+", + ], &batches); + + let batches = FileStreamTest::new() + .with_records(vec![make_partition(3), make_partition(2)]) + .with_num_files(3) + .with_on_error(OnError::Skip) + .with_open_errors(vec![0]) + .with_scan_errors(vec![1]) + .result() + .await?; + + #[rustfmt::skip] + assert_batches_eq!(&[ + "+---+", + "| i |", + "+---+", + "| 0 |", + "| 1 |", + "| 2 |", + "| 0 |", + "| 1 |", + "+---+", + ], &batches); + + let batches = FileStreamTest::new() + .with_records(vec![make_partition(3), make_partition(2)]) + .with_num_files(3) + .with_on_error(OnError::Skip) + .with_open_errors(vec![2]) + .with_scan_errors(vec![0, 1]) + .result() + .await?; + + #[rustfmt::skip] + assert_batches_eq!(&[ + "++", + "++", + ], &batches); + + let batches = FileStreamTest::new() + .with_records(vec![make_partition(3), make_partition(2)]) + .with_num_files(3) + .with_on_error(OnError::Skip) + .with_open_errors(vec![0, 2]) + .with_scan_errors(vec![1]) + .result() + .await?; + + #[rustfmt::skip] + assert_batches_eq!(&[ + "++", + "++", + ], &batches); + + Ok(()) + } + + #[tokio::test] + async fn without_limit() -> Result<()> { + let batches = create_and_collect(None).await; + + #[rustfmt::skip] + assert_batches_eq!(&[ + "+---+", + "| i |", + "+---+", + "| 0 |", + "| 1 |", + "| 2 |", + "| 0 |", + "| 1 |", + "| 0 |", + "| 1 |", + "| 2 |", + "| 0 |", + "| 1 |", + "+---+", + ], &batches); + + Ok(()) + } + + #[tokio::test] + async fn with_limit_between_files() -> Result<()> { + let batches = create_and_collect(Some(5)).await; + #[rustfmt::skip] + assert_batches_eq!(&[ + "+---+", + "| i |", + "+---+", + "| 0 |", + "| 1 |", + "| 2 |", + "| 0 |", + "| 1 |", + "+---+", + ], &batches); + + Ok(()) + } + + #[tokio::test] + async fn with_limit_at_middle_of_batch() -> Result<()> { + let batches = create_and_collect(Some(6)).await; + #[rustfmt::skip] + assert_batches_eq!(&[ + "+---+", + "| i |", + "+---+", + "| 0 |", + "| 1 |", + "| 2 |", + "| 0 |", + "| 1 |", + "| 0 |", + "+---+", + ], &batches); + + Ok(()) + } +} diff --git a/datafusion/datasource/src/memory.rs b/datafusion/datasource/src/memory.rs index efb178ad078e..182ffebdf461 100644 --- a/datafusion/datasource/src/memory.rs +++ b/datafusion/datasource/src/memory.rs @@ -765,28 +765,17 @@ mod memory_source_tests { #[cfg(test)] mod tests { + use crate::tests::{aggr_test_schema, make_partition}; + use super::*; - use arrow::array::{ArrayRef, Int32Array}; + use datafusion_physical_plan::expressions::lit; - use std::collections::HashMap; use arrow::datatypes::{DataType, Field}; use datafusion_common::assert_batches_eq; use datafusion_common::stats::{ColumnStatistics, Precision}; use futures::StreamExt; - // Return a RecordBatch with a single Int32 array with values (0..sz) in a field named "i" - pub fn make_partition(sz: i32) -> RecordBatch { - let seq_start = 0; - let seq_end = sz; - let values = (seq_start..seq_end).collect::>(); - let schema = Arc::new(Schema::new(vec![Field::new("i", DataType::Int32, true)])); - let arr = Arc::new(Int32Array::from(values)); - let arr = arr as ArrayRef; - - RecordBatch::try_new(schema, vec![arr]).unwrap() - } - #[tokio::test] async fn exec_with_limit() -> Result<()> { let task_ctx = Arc::new(TaskContext::default()); @@ -813,29 +802,6 @@ mod tests { Ok(()) } - /// Get the schema for the aggregate_test_* csv files - pub 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) - } - #[tokio::test] async fn values_empty_case() -> Result<()> { let schema = aggr_test_schema(); diff --git a/datafusion/datasource/src/mod.rs b/datafusion/datasource/src/mod.rs index 8183d7b53244..e60b02f9c9e6 100644 --- a/datafusion/datasource/src/mod.rs +++ b/datafusion/datasource/src/mod.rs @@ -24,6 +24,8 @@ //! A table that uses the `ObjectStore` listing capability //! to get the list of files to process. +pub mod display; +pub mod file; pub mod file_compression_type; pub mod file_groups; pub mod file_meta; @@ -32,6 +34,9 @@ pub mod file_sink_config; pub mod file_stream; pub mod memory; pub mod source; +mod statistics; +#[cfg(test)] +mod test_util; pub mod url; pub mod write; use chrono::TimeZone; @@ -187,13 +192,51 @@ impl From for PartitionedFile { #[cfg(test)] mod tests { use super::ListingTableUrl; + use arrow::{ + array::{ArrayRef, Int32Array, RecordBatch}, + datatypes::{DataType, Field, Schema, SchemaRef}, + }; use datafusion_execution::object_store::{ DefaultObjectStoreRegistry, ObjectStoreRegistry, }; use object_store::{local::LocalFileSystem, path::Path}; - use std::{ops::Not, sync::Arc}; + use std::{collections::HashMap, ops::Not, sync::Arc}; use url::Url; + /// Return a RecordBatch with a single Int32 array with values (0..sz) in a field named "i" + pub fn make_partition(sz: i32) -> RecordBatch { + let seq_start = 0; + let seq_end = sz; + let values = (seq_start..seq_end).collect::>(); + let schema = Arc::new(Schema::new(vec![Field::new("i", DataType::Int32, true)])); + let arr = Arc::new(Int32Array::from(values)); + + RecordBatch::try_new(schema, vec![arr as ArrayRef]).unwrap() + } + + /// Get the schema for the aggregate_test_* csv files + pub 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) + } + #[test] fn test_object_store_listing_url() { let listing = ListingTableUrl::parse("file:///").unwrap(); diff --git a/datafusion/core/src/datasource/physical_plan/statistics.rs b/datafusion/datasource/src/statistics.rs similarity index 97% rename from datafusion/core/src/datasource/physical_plan/statistics.rs rename to datafusion/datasource/src/statistics.rs index 5811c19be408..9df5aa993d43 100644 --- a/datafusion/core/src/datasource/physical_plan/statistics.rs +++ b/datafusion/datasource/src/statistics.rs @@ -15,18 +15,14 @@ // 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 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::datasource::listing::PartitionedFile; +use crate::PartitionedFile; use arrow::array::RecordBatch; use arrow::datatypes::SchemaRef; diff --git a/datafusion/datasource/src/test_util.rs b/datafusion/datasource/src/test_util.rs new file mode 100644 index 000000000000..ab025069bf76 --- /dev/null +++ b/datafusion/datasource/src/test_util.rs @@ -0,0 +1,84 @@ +// 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 std::sync::Arc; + +use arrow::datatypes::SchemaRef; +use datafusion_common::Statistics; +use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; +use object_store::ObjectStore; + +use crate::{ + file::FileSource, file_scan_config::FileScanConfig, file_stream::FileOpener, +}; +use datafusion_common::Result; + +/// Minimal [`FileSource`] implementation for use in tests. +#[derive(Clone, Default)] +pub struct MockSource { + metrics: ExecutionPlanMetricsSet, + projected_statistics: Option, +} + +impl FileSource for MockSource { + fn create_file_opener( + &self, + _object_store: Arc, + _base_config: &FileScanConfig, + _partition: usize, + ) -> Arc { + unimplemented!() + } + + fn as_any(&self) -> &dyn std::any::Any { + self + } + + fn with_batch_size(&self, _batch_size: usize) -> Arc { + Arc::new(Self { ..self.clone() }) + } + + fn with_schema(&self, _schema: SchemaRef) -> Arc { + Arc::new(Self { ..self.clone() }) + } + + fn with_projection(&self, _config: &FileScanConfig) -> Arc { + Arc::new(Self { ..self.clone() }) + } + + fn with_statistics(&self, statistics: Statistics) -> Arc { + let mut source = self.clone(); + source.projected_statistics = Some(statistics); + Arc::new(source) + } + + fn metrics(&self) -> &ExecutionPlanMetricsSet { + &self.metrics + } + + fn statistics(&self) -> Result { + Ok(self + .projected_statistics + .as_ref() + .expect("projected_statistics must be set") + .clone()) + } + + fn file_type(&self) -> &str { + "mock" + } +} diff --git a/datafusion/proto/src/physical_plan/from_proto.rs b/datafusion/proto/src/physical_plan/from_proto.rs index 34fb5bb6ddc1..ce01865b8c73 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -26,14 +26,13 @@ use object_store::path::Path; use object_store::ObjectMeta; use datafusion::arrow::datatypes::Schema; -use datafusion::datasource::data_source::FileSource; use datafusion::datasource::file_format::csv::CsvSink; use datafusion::datasource::file_format::json::JsonSink; #[cfg(feature = "parquet")] use datafusion::datasource::file_format::parquet::ParquetSink; use datafusion::datasource::listing::{FileRange, ListingTableUrl, PartitionedFile}; use datafusion::datasource::object_store::ObjectStoreUrl; -use datafusion::datasource::physical_plan::{FileScanConfig, FileSinkConfig}; +use datafusion::datasource::physical_plan::{FileScanConfig, FileSinkConfig, FileSource}; use datafusion::execution::FunctionRegistry; use datafusion::logical_expr::WindowFunctionDefinition; use datafusion::physical_expr::{LexOrdering, PhysicalSortExpr, ScalarFunctionExpr};