From 37c804142ef939305b331d973a351d90b61a4809 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Mon, 4 Oct 2021 17:59:07 +0200 Subject: [PATCH 01/17] [feat] added store --- datafusion/src/avro_to_arrow/mod.rs | 6 +- datafusion/src/datasource/file_format/avro.rs | 65 ++++++++--- datafusion/src/datasource/file_format/csv.rs | 98 ++++++++++++----- datafusion/src/datasource/file_format/json.rs | 78 ++++++++++---- datafusion/src/datasource/file_format/mod.rs | 29 +++-- .../src/datasource/file_format/parquet.rs | 102 ++++++++++++++---- datafusion/src/datasource/listing.rs | 84 ++++++++++----- .../src/datasource/object_store/local.rs | 47 +++++--- datafusion/src/datasource/object_store/mod.rs | 72 ++++++++++--- .../src/physical_plan/file_format/avro.rs | 67 ++++++++---- .../src/physical_plan/file_format/csv.rs | 46 +++++--- .../src/physical_plan/file_format/json.rs | 75 +++++++------ .../src/physical_plan/file_format/parquet.rs | 43 +++++--- 13 files changed, 582 insertions(+), 230 deletions(-) diff --git a/datafusion/src/avro_to_arrow/mod.rs b/datafusion/src/avro_to_arrow/mod.rs index 531b1092e1d62..f30fbdcc0cec2 100644 --- a/datafusion/src/avro_to_arrow/mod.rs +++ b/datafusion/src/avro_to_arrow/mod.rs @@ -28,11 +28,11 @@ use crate::arrow::datatypes::Schema; use crate::error::Result; #[cfg(feature = "avro")] pub use reader::{Reader, ReaderBuilder}; -use std::io::{Read, Seek}; +use std::io::Read; #[cfg(feature = "avro")] /// Read Avro schema given a reader -pub fn read_avro_schema_from_reader(reader: &mut R) -> Result { +pub fn read_avro_schema_from_reader(reader: &mut R) -> Result { let avro_reader = avro_rs::Reader::new(reader)?; let schema = avro_reader.writer_schema(); schema::to_arrow_schema(schema) @@ -40,7 +40,7 @@ pub fn read_avro_schema_from_reader(reader: &mut R) -> Result(_: &mut R) -> Result { +pub fn read_avro_schema_from_reader(_: &mut R) -> Result { Err(crate::error::DataFusionError::NotImplemented( "cannot read avro schema without the 'avro' feature enabled".to_string(), )) diff --git a/datafusion/src/datasource/file_format/avro.rs b/datafusion/src/datasource/file_format/avro.rs index bb5083d547b7b..c6f7b24f0e7db 100644 --- a/datafusion/src/datasource/file_format/avro.rs +++ b/datafusion/src/datasource/file_format/avro.rs @@ -23,11 +23,11 @@ use arrow::datatypes::Schema; use arrow::{self, datatypes::SchemaRef}; use async_trait::async_trait; use futures::StreamExt; -use std::fs::File; +use super::FileFormat; use super::PartitionedFile; -use super::{FileFormat, StringStream}; use crate::avro_to_arrow::read_avro_schema_from_reader; +use crate::datasource::object_store::{ObjectStoreRegistry, SizedFile, SizedFileStream}; use crate::error::Result; use crate::logical_plan::Expr; use crate::physical_plan::file_format::AvroExec; @@ -35,22 +35,46 @@ use crate::physical_plan::ExecutionPlan; use crate::physical_plan::Statistics; /// Line-delimited Avro `FileFormat` implementation. -pub struct AvroFormat {} +pub struct AvroFormat { + /// Object store registry + pub object_store_registry: Arc, +} + +impl Default for AvroFormat { + fn default() -> Self { + Self { + object_store_registry: Arc::new(ObjectStoreRegistry::new()), + } + } +} + +impl AvroFormat { + /// Create Parquet with the given object store and default values + pub fn new(object_store_registry: Arc) -> Self { + Self { + object_store_registry, + } + } +} #[async_trait] impl FileFormat for AvroFormat { - async fn infer_schema(&self, mut paths: StringStream) -> Result { + async fn infer_schema(&self, mut file_stream: SizedFileStream) -> Result { let mut schemas = vec![]; - while let Some(filename) = paths.next().await { - let mut file = File::open(filename)?; - let schema = read_avro_schema_from_reader(&mut file)?; + while let Some(fmeta_res) = file_stream.next().await { + let fmeta = fmeta_res?; + let fsize = fmeta.size as usize; + let object_store = self.object_store_registry.get_by_uri(&fmeta.path)?; + let obj_reader = object_store.file_reader(fmeta)?; + let mut reader = obj_reader.chunk_reader(0, fsize)?; + let schema = read_avro_schema_from_reader(&mut reader)?; schemas.push(schema); } let merged_schema = Schema::try_merge(schemas)?; Ok(Arc::new(merged_schema)) } - async fn infer_stats(&self, _path: &str) -> Result { + async fn infer_stats(&self, _path: SizedFile) -> Result { Ok(Statistics::default()) } @@ -65,8 +89,9 @@ impl FileFormat for AvroFormat { limit: Option, ) -> Result> { let exec = AvroExec::new( + Arc::clone(&self.object_store_registry), // flattening this for now because CsvExec does not support partitioning yet - files.into_iter().flatten().map(|f| f.path).collect(), + files.into_iter().flatten().collect(), statistics, schema, projection.clone(), @@ -75,13 +100,19 @@ impl FileFormat for AvroFormat { ); Ok(Arc::new(exec)) } + + fn object_store_registry(&self) -> &Arc { + &self.object_store_registry + } } #[cfg(test)] #[cfg(feature = "avro")] mod tests { - use crate::datasource::file_format::string_stream; - use crate::physical_plan::collect; + use crate::{ + datasource::object_store::local::{local_sized_file, local_sized_file_stream}, + physical_plan::collect, + }; use super::*; use arrow::array::{ @@ -327,16 +358,18 @@ mod tests { ) -> Result> { let testdata = crate::test_util::arrow_test_data(); let filename = format!("{}/avro/{}", testdata, file_name); - let format = AvroFormat {}; + let format = AvroFormat::default(); let schema = format - .infer_schema(string_stream(vec![filename.clone()])) + .infer_schema(local_sized_file_stream(vec![filename.clone()])) .await .expect("Schema inference"); let stats = format - .infer_stats(&filename) + .infer_stats(local_sized_file(filename.clone())) .await .expect("Stats inference"); - let files = vec![vec![PartitionedFile { path: filename }]]; + let files = vec![vec![PartitionedFile { + file: local_sized_file(filename.to_owned()), + }]]; let exec = format .create_physical_plan(schema, files, stats, projection, batch_size, &[], None) .await?; @@ -356,7 +389,7 @@ mod tests { async fn test() -> Result<()> { let testdata = crate::test_util::arrow_test_data(); let filename = format!("{}/avro/alltypes_plain.avro", testdata); - let schema_result = AvroFormat {} + let schema_result = AvroFormat::default() .infer_schema(string_stream(vec![filename])) .await; assert!(matches!( diff --git a/datafusion/src/datasource/file_format/csv.rs b/datafusion/src/datasource/file_format/csv.rs index a5b555053ef76..288c9c6faa6fa 100644 --- a/datafusion/src/datasource/file_format/csv.rs +++ b/datafusion/src/datasource/file_format/csv.rs @@ -23,10 +23,10 @@ use arrow::datatypes::Schema; use arrow::{self, datatypes::SchemaRef}; use async_trait::async_trait; use futures::StreamExt; -use std::fs::File; +use super::FileFormat; use super::PartitionedFile; -use super::{FileFormat, StringStream}; +use crate::datasource::object_store::{ObjectStoreRegistry, SizedFile, SizedFileStream}; use crate::error::Result; use crate::logical_plan::Expr; use crate::physical_plan::file_format::CsvExec; @@ -35,25 +35,69 @@ use crate::physical_plan::Statistics; /// Character Separated Value `FileFormat` implementation. pub struct CsvFormat { + has_header: bool, + delimiter: u8, + schema_infer_max_rec: Option, + object_store_registry: Arc, +} + +impl Default for CsvFormat { + fn default() -> Self { + Self { + object_store_registry: Arc::new(ObjectStoreRegistry::new()), + schema_infer_max_rec: None, + has_header: true, + delimiter: b',', + } + } +} + +impl CsvFormat { + /// Create Parquet with the given object store and default values + pub fn new(object_store_registry: Arc) -> Self { + Self { + object_store_registry, + ..Default::default() + } + } + + /// Set a limit in terms of records to scan to infer the schema + /// - default to `None` (no limit) + pub fn with_schema_infer_max_rec(&mut self, max_rec: Option) -> &mut Self { + self.schema_infer_max_rec = max_rec; + self + } + /// Set true to indicate that the first line is a header. - pub has_header: bool, - /// The character seprating values within a row. - pub delimiter: u8, - /// If no schema was provided for the table, it will be - /// infered from the data itself, this limits the number - /// of lines used in the process. - pub schema_infer_max_rec: Option, + /// - default to true + pub fn with_has_header(&mut self, has_header: bool) -> &mut Self { + self.has_header = has_header; + self + } + + /// The character separating values within a row. + /// - default to ',' + pub fn with_delimiter(&mut self, delimiter: u8) -> &mut Self { + self.delimiter = delimiter; + self + } } #[async_trait] impl FileFormat for CsvFormat { - async fn infer_schema(&self, mut paths: StringStream) -> Result { + async fn infer_schema(&self, mut file_stream: SizedFileStream) -> Result { let mut schemas = vec![]; let mut records_to_read = self.schema_infer_max_rec.unwrap_or(std::usize::MAX); - while let Some(fname) = paths.next().await { - let (schema, records_read) = arrow::csv::reader::infer_file_schema( - &mut File::open(fname)?, + while let Some(fmeta_res) = file_stream.next().await { + let fmeta = fmeta_res?; + let fsize = fmeta.size as usize; + let object_store = self.object_store_registry.get_by_uri(&fmeta.path)?; + + let obj_reader = object_store.file_reader(fmeta)?; + let mut reader = obj_reader.chunk_reader(0, fsize)?; + let (schema, records_read) = arrow::csv::reader::infer_reader_schema( + &mut reader, self.delimiter, Some(records_to_read), self.has_header, @@ -72,7 +116,7 @@ impl FileFormat for CsvFormat { Ok(Arc::new(merged_schema)) } - async fn infer_stats(&self, _path: &str) -> Result { + async fn infer_stats(&self, _path: SizedFile) -> Result { Ok(Statistics::default()) } @@ -87,8 +131,9 @@ impl FileFormat for CsvFormat { limit: Option, ) -> Result> { let exec = CsvExec::new( + Arc::clone(&self.object_store_registry), // flattening this for now because CsvExec does not support partitioning yet - files.into_iter().flatten().map(|f| f.path).collect(), + files.into_iter().flatten().collect(), statistics, schema, self.has_header, @@ -99,6 +144,10 @@ impl FileFormat for CsvFormat { ); Ok(Arc::new(exec)) } + + fn object_store_registry(&self) -> &Arc { + &self.object_store_registry + } } #[cfg(test)] @@ -106,7 +155,10 @@ mod tests { use arrow::array::StringArray; use super::*; - use crate::{datasource::file_format::string_stream, physical_plan::collect}; + use crate::{ + datasource::object_store::local::{local_sized_file, local_sized_file_stream}, + physical_plan::collect, + }; #[tokio::test] async fn read_small_batches() -> Result<()> { @@ -199,20 +251,18 @@ mod tests { ) -> Result> { let testdata = crate::test_util::arrow_test_data(); let filename = format!("{}/csv/{}", testdata, file_name); - let format = CsvFormat { - has_header: true, - schema_infer_max_rec: Some(1000), - delimiter: b',', - }; + let format = CsvFormat::default(); let schema = format - .infer_schema(string_stream(vec![filename.clone()])) + .infer_schema(local_sized_file_stream(vec![filename.clone()])) .await .expect("Schema inference"); let stats = format - .infer_stats(&filename) + .infer_stats(local_sized_file(filename.clone())) .await .expect("Stats inference"); - let files = vec![vec![PartitionedFile { path: filename }]]; + let files = vec![vec![PartitionedFile { + file: local_sized_file(filename.to_owned()), + }]]; let exec = format .create_physical_plan(schema, files, stats, projection, batch_size, &[], None) .await?; diff --git a/datafusion/src/datasource/file_format/json.rs b/datafusion/src/datasource/file_format/json.rs index 7357644af1fa3..02f5fcd5a9557 100644 --- a/datafusion/src/datasource/file_format/json.rs +++ b/datafusion/src/datasource/file_format/json.rs @@ -26,10 +26,12 @@ use arrow::json::reader::infer_json_schema_from_iterator; use arrow::json::reader::ValueIter; use async_trait::async_trait; use futures::StreamExt; -use std::fs::File; +use super::FileFormat; use super::PartitionedFile; -use super::{FileFormat, StringStream}; +use crate::datasource::object_store::ObjectStoreRegistry; +use crate::datasource::object_store::SizedFile; +use crate::datasource::object_store::SizedFileStream; use crate::error::Result; use crate::logical_plan::Expr; use crate::physical_plan::file_format::NdJsonExec; @@ -38,20 +40,49 @@ use crate::physical_plan::Statistics; /// New line delimited JSON `FileFormat` implementation. pub struct JsonFormat { - /// If no schema was provided for the table, it will be - /// infered from the data itself, this limits the number - /// of lines used in the process. - pub schema_infer_max_rec: Option, + schema_infer_max_rec: Option, + object_store_registry: Arc, +} + +impl Default for JsonFormat { + fn default() -> Self { + Self { + schema_infer_max_rec: None, + object_store_registry: Arc::new(ObjectStoreRegistry::new()), + } + } +} + +impl JsonFormat { + /// Create Parquet with the given object store and default values + pub fn new(object_store_registry: Arc) -> Self { + Self { + object_store_registry, + ..Default::default() + } + } + + /// Set a limit in terms of records to scan to infer the schema + /// - defaults to `None` (no limit) + pub fn with_schema_infer_max_rec(&mut self, max_rec: Option) -> &mut Self { + self.schema_infer_max_rec = max_rec; + self + } } #[async_trait] impl FileFormat for JsonFormat { - async fn infer_schema(&self, mut paths: StringStream) -> Result { + async fn infer_schema(&self, mut file_stream: SizedFileStream) -> Result { let mut schemas = Vec::new(); let mut records_to_read = self.schema_infer_max_rec.unwrap_or(usize::MAX); - while let Some(file) = paths.next().await { - let file = File::open(file)?; - let mut reader = BufReader::new(file); + while let Some(fmeta_res) = file_stream.next().await { + let fmeta = fmeta_res?; + let fsize = fmeta.size as usize; + let object_store = self.object_store_registry.get_by_uri(&fmeta.path)?; + + let obj_reader = object_store.file_reader(fmeta)?; + let chunk_reader = obj_reader.chunk_reader(0, fsize)?; + let mut reader = BufReader::new(chunk_reader); let iter = ValueIter::new(&mut reader, None); let schema = infer_json_schema_from_iterator(iter.take_while(|_| { let should_take = records_to_read > 0; @@ -68,7 +99,7 @@ impl FileFormat for JsonFormat { Ok(Arc::new(schema)) } - async fn infer_stats(&self, _path: &str) -> Result { + async fn infer_stats(&self, _path: SizedFile) -> Result { Ok(Statistics::default()) } @@ -83,8 +114,9 @@ impl FileFormat for JsonFormat { limit: Option, ) -> Result> { let exec = NdJsonExec::new( + Arc::clone(&self.object_store_registry), // flattening this for now because NdJsonExec does not support partitioning yet - files.into_iter().flatten().map(|f| f.path).collect(), + files.into_iter().flatten().collect(), statistics, schema, projection.clone(), @@ -93,6 +125,10 @@ impl FileFormat for JsonFormat { ); Ok(Arc::new(exec)) } + + fn object_store_registry(&self) -> &Arc { + &self.object_store_registry + } } #[cfg(test)] @@ -100,7 +136,10 @@ mod tests { use arrow::array::Int64Array; use super::*; - use crate::{datasource::file_format::string_stream, physical_plan::collect}; + use crate::{ + datasource::object_store::local::{local_sized_file, local_sized_file_stream}, + physical_plan::collect, + }; #[tokio::test] async fn read_small_batches() -> Result<()> { @@ -176,16 +215,17 @@ mod tests { batch_size: usize, ) -> Result> { let filename = "tests/jsons/2.json"; - let format = JsonFormat { - schema_infer_max_rec: Some(1000), - }; + let format = JsonFormat::default(); let schema = format - .infer_schema(string_stream(vec![filename.to_owned()])) + .infer_schema(local_sized_file_stream(vec![filename.to_owned()])) .await .expect("Schema inference"); - let stats = format.infer_stats(filename).await.expect("Stats inference"); + let stats = format + .infer_stats(local_sized_file(filename.to_owned())) + .await + .expect("Stats inference"); let files = vec![vec![PartitionedFile { - path: filename.to_owned(), + file: local_sized_file(filename.to_owned()), }]]; let exec = format .create_physical_plan(schema, files, stats, projection, batch_size, &[], None) diff --git a/datafusion/src/datasource/file_format/mod.rs b/datafusion/src/datasource/file_format/mod.rs index e08ebd615c802..8ce30a5a9a151 100644 --- a/datafusion/src/datasource/file_format/mod.rs +++ b/datafusion/src/datasource/file_format/mod.rs @@ -34,13 +34,15 @@ use crate::physical_plan::{Accumulator, ExecutionPlan, Statistics}; use async_trait::async_trait; use futures::Stream; -/// A stream of String that can be used accross await calls -pub type StringStream = Pin + Send + Sync>>; +use super::object_store::{ObjectStoreRegistry, SizedFile, SizedFileStream}; -/// Convert a vector into a stream -pub fn string_stream(strings: Vec) -> StringStream { - Box::pin(futures::stream::iter(strings)) -} +// /// A stream of String that can be used accross await calls +// pub type StringStream = Pin + Send + Sync>>; + +// /// Convert a vector into a stream +// pub fn string_stream(strings: Vec) -> StringStream { +// Box::pin(futures::stream::iter(strings)) +// } /// This trait abstracts all the file format specific implementations /// from the `TableProvider`. This helps code re-utilization accross @@ -49,10 +51,10 @@ pub fn string_stream(strings: Vec) -> StringStream { pub trait FileFormat: Send + Sync { /// Open the files at the paths provided by iterator and infer the /// common schema - async fn infer_schema(&self, paths: StringStream) -> Result; + async fn infer_schema(&self, paths: SizedFileStream) -> Result; /// Open the file at the given path and infer its statistics - async fn infer_stats(&self, path: &str) -> Result; + async fn infer_stats(&self, path: SizedFile) -> Result; /// Take a list of files and convert it to the appropriate executor /// according to this file format. @@ -69,6 +71,9 @@ pub trait FileFormat: Send + Sync { filters: &[Expr], limit: Option, ) -> Result>; + + /// Get the oject store from which to read this file format + fn object_store_registry(&self) -> &Arc; } /// Get all files as well as the summary statistic @@ -162,15 +167,19 @@ pub fn get_statistics_with_limit( /// TODO move back to crate::datasource::mod.rs once legacy cleaned up pub struct PartitionedFile { /// Path for the file (e.g. URL, filesystem path, etc) - pub path: String, + pub file: SizedFile, // Values of partition columns to be appended to each row // pub partition_value: Option>, // We may include row group range here for a more fine-grained parallel execution } +/// Stream of files get listed from object store +pub type PartitionedFileStream = + Pin> + Send + Sync + 'static>>; + impl std::fmt::Display for PartitionedFile { fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { - write!(f, "{}", self.path) + write!(f, "{}", self.file) } } diff --git a/datafusion/src/datasource/file_format/parquet.rs b/datafusion/src/datasource/file_format/parquet.rs index 39aa14de0b9f0..26b49d76cc97d 100644 --- a/datafusion/src/datasource/file_format/parquet.rs +++ b/datafusion/src/datasource/file_format/parquet.rs @@ -17,7 +17,7 @@ //! Parquet format abstractions -use std::fs::File; +use std::io::Read; use std::sync::Arc; use arrow::datatypes::Schema; @@ -26,13 +26,21 @@ use async_trait::async_trait; use futures::stream::StreamExt; use parquet::arrow::ArrowReader; use parquet::arrow::ParquetFileArrowReader; +use parquet::errors::ParquetError; +use parquet::errors::Result as ParquetResult; +use parquet::file::reader::ChunkReader; +use parquet::file::reader::Length; use parquet::file::serialized_reader::SerializedFileReader; use parquet::file::statistics::Statistics as ParquetStatistics; use super::FileFormat; use super::PartitionedFile; -use super::{create_max_min_accs, get_col_stats, StringStream}; +use super::{create_max_min_accs, get_col_stats}; use crate::arrow::datatypes::{DataType, Field}; +use crate::datasource::object_store::ObjectReader; +use crate::datasource::object_store::ObjectStoreRegistry; +use crate::datasource::object_store::SizedFile; +use crate::datasource::object_store::SizedFileStream; use crate::error::DataFusionError; use crate::error::Result; use crate::logical_plan::combine_filters; @@ -45,26 +53,52 @@ use crate::scalar::ScalarValue; /// The Apache Parquet `FileFormat` implementation pub struct ParquetFormat { + object_store_registry: Arc, + enable_pruning: bool, +} + +impl Default for ParquetFormat { + fn default() -> Self { + Self { + enable_pruning: true, + object_store_registry: Arc::new(ObjectStoreRegistry::new()), + } + } +} + +impl ParquetFormat { + /// Create Parquet with the given object store and default values + pub fn new(object_store_registry: Arc) -> Self { + Self { + object_store_registry, + ..Default::default() + } + } + /// Activate statistics based row group level pruning - pub enable_pruning: bool, + /// - defaults to true + pub fn with_enable_pruning(&mut self, enable: bool) -> &mut Self { + self.enable_pruning = enable; + self + } } #[async_trait] impl FileFormat for ParquetFormat { - async fn infer_schema(&self, mut paths: StringStream) -> Result { + async fn infer_schema(&self, mut paths: SizedFileStream) -> Result { // We currently get the schema information from the first file rather than do // schema merging and this is a limitation. // See https://issues.apache.org/jira/browse/ARROW-11017 let first_file = paths .next() .await - .ok_or_else(|| DataFusionError::Plan("No data file found".to_owned()))?; - let (schema, _) = fetch_metadata(&first_file)?; + .ok_or_else(|| DataFusionError::Plan("No data file found".to_owned()))??; + let (schema, _) = fetch_metadata(&self.object_store_registry, first_file)?; Ok(Arc::new(schema)) } - async fn infer_stats(&self, path: &str) -> Result { - let (_, stats) = fetch_metadata(path)?; + async fn infer_stats(&self, path: SizedFile) -> Result { + let (_, stats) = fetch_metadata(&self.object_store_registry, path)?; Ok(stats) } @@ -88,6 +122,7 @@ impl FileFormat for ParquetFormat { }; Ok(Arc::new(ParquetExec::new( + Arc::clone(&self.object_store_registry), files, statistics, schema, @@ -99,6 +134,10 @@ impl FileFormat for ParquetFormat { limit, ))) } + + fn object_store_registry(&self) -> &Arc { + &self.object_store_registry + } } fn summarize_min_max( @@ -224,9 +263,13 @@ fn summarize_min_max( } /// Read and parse the metadata of the Parquet file at location `path` -fn fetch_metadata(path: &str) -> Result<(Schema, Statistics)> { - let file = File::open(path)?; - let file_reader = Arc::new(SerializedFileReader::new(file)?); +fn fetch_metadata( + object_store_registry: &ObjectStoreRegistry, + fmeta: SizedFile, +) -> Result<(Schema, Statistics)> { + let object_store = object_store_registry.get_by_uri(&fmeta.path)?; + let obj_reader = ChunkObjectReader(object_store.file_reader(fmeta)?); + let file_reader = Arc::new(SerializedFileReader::new(obj_reader)?); let mut arrow_reader = ParquetFileArrowReader::new(file_reader); let schema = arrow_reader.get_schema()?; let num_fields = schema.fields().len(); @@ -282,10 +325,31 @@ fn fetch_metadata(path: &str) -> Result<(Schema, Statistics)> { Ok((schema, statistics)) } +/// A wrapper around the object reader to make it implement `ChunkReader` +pub struct ChunkObjectReader(pub Arc); + +impl Length for ChunkObjectReader { + fn len(&self) -> u64 { + self.0.length() + } +} + +impl ChunkReader for ChunkObjectReader { + type T = Box; + + fn get_read(&self, start: u64, length: usize) -> ParquetResult { + self.0 + .chunk_reader(start, length) + .map_err(|e| ParquetError::ArrowError(e.to_string())) + } +} + #[cfg(test)] mod tests { - use crate::datasource::file_format::string_stream; - use crate::physical_plan::collect; + use crate::{ + datasource::object_store::local::{local_sized_file, local_sized_file_stream}, + physical_plan::collect, + }; use super::*; use arrow::array::{ @@ -521,18 +585,18 @@ mod tests { ) -> Result> { let testdata = crate::test_util::parquet_test_data(); let filename = format!("{}/{}", testdata, file_name); - let format = ParquetFormat { - enable_pruning: true, - }; + let format = ParquetFormat::default(); let schema = format - .infer_schema(string_stream(vec![filename.clone()])) + .infer_schema(local_sized_file_stream(vec![filename.clone()])) .await .expect("Schema inference"); let stats = format - .infer_stats(&filename.clone()) + .infer_stats(local_sized_file(filename.clone())) .await .expect("Stats inference"); - let files = vec![vec![PartitionedFile { path: filename }]]; + let files = vec![vec![PartitionedFile { + file: local_sized_file(filename.clone()), + }]]; let exec = format .create_physical_plan(schema, files, stats, projection, batch_size, &[], None) .await?; diff --git a/datafusion/src/datasource/listing.rs b/datafusion/src/datasource/listing.rs index cac9b36b3d51f..3798aaff46868 100644 --- a/datafusion/src/datasource/listing.rs +++ b/datafusion/src/datasource/listing.rs @@ -28,11 +28,14 @@ use crate::{ datasource::file_format::{self, PartitionedFile}, error::Result, logical_plan::Expr, - physical_plan::{common, ExecutionPlan, Statistics}, + physical_plan::{ExecutionPlan, Statistics}, }; use super::{ - datasource::TableProviderFilterPushDown, file_format::FileFormat, TableProvider, + datasource::TableProviderFilterPushDown, + file_format::{FileFormat, PartitionedFileStream}, + object_store::ObjectStoreRegistry, + TableProvider, }; /// Options for creating a `ListingTable` @@ -67,9 +70,11 @@ impl ListingOptions { /// This way when creating the logical plan we can decide to resolve the schema /// locally or ask a remote service to do it (e.g a scheduler). pub async fn infer_schema(&self, path: &str) -> Result { - let files = - futures::stream::iter(common::build_file_list(path, &self.file_extension)?); - let file_schema = self.format.infer_schema(Box::pin(files)).await?; + let object_store = self.format.object_store_registry().get_by_uri(path)?; + let file_stream = object_store + .list_file_with_suffix(path, &self.file_extension) + .await?; + let file_schema = self.format.infer_schema(file_stream).await?; // Add the partition columns to the file schema let mut fields = file_schema.fields().clone(); for part in &self.partitions { @@ -123,21 +128,27 @@ impl TableProvider for ListingTable { ) -> Result> { // list files (with partitions) let file_list = pruned_partition_list( + &self.options.format.object_store_registry(), &self.path, filters, &self.options.file_extension, &self.options.partitions, - )?; + ) + .await?; // collect the statistics if required by the config - let files = futures::stream::iter(file_list) - .then(|file| async { + let files = file_list + .then(|part_file| async { + let part_file = part_file?; let statistics = if self.options.collect_stat { - self.options.format.infer_stats(&file.path).await? + self.options + .format + .infer_stats(part_file.file.clone()) + .await? } else { Statistics::default() }; - Ok((file, statistics)) as Result<(PartitionedFile, Statistics)> + Ok((part_file, statistics)) as Result<(PartitionedFile, Statistics)> }) .try_collect::>() .await?; @@ -172,21 +183,21 @@ impl TableProvider for ListingTable { /// Discover the partitions on the given path and prune out files /// relative to irrelevant partitions using `filters` expressions -fn pruned_partition_list( - // registry: &ObjectStoreRegistry, +async fn pruned_partition_list( + registry: &ObjectStoreRegistry, path: &str, _filters: &[Expr], file_extension: &str, partition_names: &[String], -) -> Result> { - let list_all = || { - Ok(common::build_file_list(path, file_extension)? - .into_iter() - .map(|f| PartitionedFile { path: f }) - .collect::>()) - }; +) -> Result { if partition_names.is_empty() { - list_all() + Ok(Box::pin( + registry + .get_by_uri(path)? + .list_file_with_suffix(path, file_extension) + .await? + .map(|f| Ok(PartitionedFile { file: f? })), + )) } else { todo!("use filters to prune partitions") } @@ -208,25 +219,44 @@ fn split_files( #[cfg(test)] mod tests { + use crate::datasource::{ + file_format::parquet::ParquetFormat, object_store::SizedFile, + }; + use super::*; #[test] fn test_split_files() { let files = vec![ PartitionedFile { - path: "a".to_owned(), + file: SizedFile { + path: "a".to_owned(), + size: 10, + }, }, PartitionedFile { - path: "b".to_owned(), + file: SizedFile { + path: "b".to_owned(), + size: 10, + }, }, PartitionedFile { - path: "c".to_owned(), + file: SizedFile { + path: "c".to_owned(), + size: 10, + }, }, PartitionedFile { - path: "d".to_owned(), + file: SizedFile { + path: "d".to_owned(), + size: 10, + }, }, PartitionedFile { - path: "e".to_owned(), + file: SizedFile { + path: "e".to_owned(), + size: 10, + }, }, ]; @@ -282,9 +312,7 @@ mod tests { let filename = format!("{}/{}", testdata, name); let opt = ListingOptions { file_extension: "parquet".to_owned(), - format: Arc::new(file_format::parquet::ParquetFormat { - enable_pruning: true, - }), + format: Arc::new(ParquetFormat::default()), partitions: vec![], max_partitions: 2, collect_stat: true, diff --git a/datafusion/src/datasource/object_store/local.rs b/datafusion/src/datasource/object_store/local.rs index 2b27f6c8f993b..18e74e2e11618 100644 --- a/datafusion/src/datasource/object_store/local.rs +++ b/datafusion/src/datasource/object_store/local.rs @@ -17,14 +17,15 @@ //! Object store that represents the Local File System. -use std::fs::Metadata; +use std::fs::{self, File, Metadata}; +use std::io::{Read, Seek, SeekFrom}; use std::sync::Arc; use async_trait::async_trait; -use futures::{stream, AsyncRead, StreamExt}; +use futures::{stream, StreamExt}; use crate::datasource::object_store::{ - FileMeta, FileMetaStream, ListEntryStream, ObjectReader, ObjectStore, + ListEntryStream, ObjectReader, ObjectStore, SizedFile, SizedFileStream, }; use crate::error::DataFusionError; use crate::error::Result; @@ -35,7 +36,7 @@ pub struct LocalFileSystem; #[async_trait] impl ObjectStore for LocalFileSystem { - async fn list_file(&self, prefix: &str) -> Result { + async fn list_file(&self, prefix: &str) -> Result { list_all(prefix.to_owned()).await } @@ -47,29 +48,31 @@ impl ObjectStore for LocalFileSystem { todo!() } - fn file_reader(&self, file: FileMeta) -> Result> { + fn file_reader(&self, file: SizedFile) -> Result> { Ok(Arc::new(LocalFileReader::new(file)?)) } } struct LocalFileReader { - file: FileMeta, + file: SizedFile, } impl LocalFileReader { - fn new(file: FileMeta) -> Result { + fn new(file: SizedFile) -> Result { Ok(Self { file }) } } #[async_trait] impl ObjectReader for LocalFileReader { - async fn chunk_reader( + fn chunk_reader( &self, - _start: u64, + start: u64, _length: usize, - ) -> Result> { - todo!() + ) -> Result> { + let mut file = File::open(&self.file.path)?; + file.seek(SeekFrom::Start(start))?; + Ok(Box::new(file)) } fn length(&self) -> u64 { @@ -77,11 +80,10 @@ impl ObjectReader for LocalFileReader { } } -async fn list_all(prefix: String) -> Result { - fn get_meta(path: String, metadata: Metadata) -> FileMeta { - FileMeta { +async fn list_all(prefix: String) -> Result { + fn get_meta(path: String, metadata: Metadata) -> SizedFile { + SizedFile { path, - last_modified: metadata.modified().map(chrono::DateTime::from).ok(), size: metadata.len(), } } @@ -89,7 +91,7 @@ async fn list_all(prefix: String) -> Result { async fn find_files_in_dir( path: String, to_visit: &mut Vec, - ) -> Result> { + ) -> Result> { let mut dir = tokio::fs::read_dir(path).await?; let mut files = Vec::new(); @@ -133,6 +135,19 @@ async fn list_all(prefix: String) -> Result { } } +/// Create a stream of `SizedFile` applying `local_sized_file` to each path +pub fn local_sized_file_stream(files: Vec) -> SizedFileStream { + Box::pin(futures::stream::iter(files).map(|f| Ok(local_sized_file(f)))) +} + +/// Helper method to fetch the file size at given path and create a `SizedFile` +pub fn local_sized_file(file: String) -> SizedFile { + SizedFile { + size: fs::metadata(&file).expect("Local file metadata").len(), + path: file, + } +} + #[cfg(test)] mod tests { use super::*; diff --git a/datafusion/src/datasource/object_store/mod.rs b/datafusion/src/datasource/object_store/mod.rs index fd25fd43a2e7d..08bbad020eb5c 100644 --- a/datafusion/src/datasource/object_store/mod.rs +++ b/datafusion/src/datasource/object_store/mod.rs @@ -20,24 +20,29 @@ pub mod local; use std::collections::HashMap; -use std::fmt::Debug; +use std::fmt::{self, Debug}; +use std::io::Read; use std::pin::Pin; use std::sync::{Arc, RwLock}; use async_trait::async_trait; -use futures::{AsyncRead, Stream}; +use futures::{Stream, StreamExt}; use local::LocalFileSystem; use crate::error::{DataFusionError, Result}; -use chrono::Utc; -/// Object Reader for one file in a object store +/** +Object Reader for one file in an object store +*/ #[async_trait] pub trait ObjectReader { /// Get reader for a part [start, start + length] in the file asynchronously - async fn chunk_reader(&self, start: u64, length: usize) - -> Result>; + fn chunk_reader( + &self, + start: u64, + length: usize, + ) -> Result>; /// Get length for the file fn length(&self) -> u64; @@ -47,25 +52,29 @@ pub trait ObjectReader { #[derive(Debug)] pub enum ListEntry { /// File metadata - FileMeta(FileMeta), + SizedFile(SizedFile), /// Prefix to be further resolved during partition discovery Prefix(String), } /// File meta we got from object store -#[derive(Debug)] -pub struct FileMeta { +#[derive(Debug, Clone)] +pub struct SizedFile { /// Path of the file pub path: String, - /// Last time the file was modified in UTC - pub last_modified: Option>, /// File size in total pub size: u64, } +impl std::fmt::Display for SizedFile { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + write!(f, "{} (size: {})", self.path, self.size) + } +} + /// Stream of files get listed from object store -pub type FileMetaStream = - Pin> + Send + Sync + 'static>>; +pub type SizedFileStream = + Pin> + Send + Sync + 'static>>; /// Stream of list entries get from object store pub type ListEntryStream = @@ -76,7 +85,24 @@ pub type ListEntryStream = #[async_trait] pub trait ObjectStore: Sync + Send + Debug { /// Returns all the files in path `prefix` - async fn list_file(&self, prefix: &str) -> Result; + async fn list_file(&self, prefix: &str) -> Result; + + /// Calls `list_file` with a suffix filter + async fn list_file_with_suffix( + &self, + prefix: &str, + suffix: &str, + ) -> Result { + let file_stream = self.list_file(prefix).await?; + let suffix = suffix.to_owned(); + Ok(Box::pin(file_stream.filter(move |fr| { + let has_suffix = match fr { + Ok(f) => f.path.ends_with(&suffix), + Err(_) => true, + }; + async move { has_suffix } + }))) + } /// Returns all the files in `prefix` if the `prefix` is already a leaf dir, /// or all paths between the `prefix` and the first occurrence of the `delimiter` if it is provided. @@ -87,7 +113,7 @@ pub trait ObjectStore: Sync + Send + Debug { ) -> Result; /// Get object reader for one file - fn file_reader(&self, file: FileMeta) -> Result>; + fn file_reader(&self, file: SizedFile) -> Result>; } static LOCAL_SCHEME: &str = "file"; @@ -100,6 +126,22 @@ pub struct ObjectStoreRegistry { pub object_stores: RwLock>>, } +impl fmt::Debug for ObjectStoreRegistry { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.debug_struct("ObjectStoreRegistry") + .field( + "schemes", + &self + .object_stores + .read() + .unwrap() + .keys() + .collect::>(), + ) + .finish() + } +} + impl ObjectStoreRegistry { /// Create the registry that object stores can registered into. /// ['LocalFileSystem'] store is registered in by default to support read local files natively. diff --git a/datafusion/src/physical_plan/file_format/avro.rs b/datafusion/src/physical_plan/file_format/avro.rs index fd50f18bf1f09..5259780e16ea2 100644 --- a/datafusion/src/physical_plan/file_format/avro.rs +++ b/datafusion/src/physical_plan/file_format/avro.rs @@ -16,6 +16,8 @@ // under the License. //! Execution plan for reading line-delimited Avro files +use crate::datasource::file_format::PartitionedFile; +use crate::datasource::object_store::ObjectStoreRegistry; use crate::error::{DataFusionError, Result}; #[cfg(feature = "avro")] use crate::physical_plan::RecordBatchStream; @@ -32,7 +34,6 @@ use std::any::Any; use std::sync::Arc; #[cfg(feature = "avro")] use std::{ - fs::File, io::Read, pin::Pin, task::{Context, Poll}, @@ -41,7 +42,8 @@ use std::{ /// Execution plan for scanning Avro data source #[derive(Debug, Clone)] pub struct AvroExec { - files: Vec, + object_store_registry: Arc, + files: Vec, statistics: Statistics, schema: SchemaRef, projection: Option>, @@ -54,7 +56,8 @@ impl AvroExec { /// Create a new JSON reader execution plan provided file list and schema /// TODO: support partitiond file list (Vec>) pub fn new( - files: Vec, + object_store_registry: Arc, + files: Vec, statistics: Statistics, schema: SchemaRef, projection: Option>, @@ -69,6 +72,7 @@ impl AvroExec { }; Self { + object_store_registry, files, statistics, schema, @@ -121,21 +125,27 @@ impl ExecutionPlan for AvroExec { #[cfg(feature = "avro")] async fn execute(&self, partition: usize) -> Result { - let mut builder = crate::avro_to_arrow::ReaderBuilder::new() - .with_schema(self.schema.clone()) - .with_batch_size(self.batch_size); - if let Some(proj) = &self.projection { - builder = builder.with_projection( - proj.iter() - .map(|col_idx| self.schema.field(*col_idx).name()) - .cloned() - .collect(), - ); - } - - let file = File::open(&self.files[partition])?; - - Ok(Box::pin(AvroStream::new(builder.build(file)?, self.limit))) + let file = self + .object_store_registry + .get_by_uri(&self.files[partition].file.path)? + .file_reader(self.files[partition].file.clone())? + .chunk_reader(0, self.files[partition].file.size as usize)?; + + let proj = self.projection.as_ref().map(|p| { + p.iter() + .map(|col_idx| self.schema.field(*col_idx).name()) + .cloned() + .collect() + }); + + let avro_reader = crate::avro_to_arrow::Reader::try_new( + file, + self.schema(), + self.batch_size, + proj, + )?; + + Ok(Box::pin(AvroStream::new(avro_reader, self.limit))) } fn fmt_as( @@ -147,10 +157,14 @@ impl ExecutionPlan for AvroExec { DisplayFormatType::Default => { write!( f, - "AvroExec: batch_size={}, limit={:?}, partitions=[{}]", + "AvroExec: batch_size={}, limit={:?}, files=[{}]", self.batch_size, self.limit, - self.files.join(", ") + self.files + .iter() + .map(|f| f.file.path.as_str()) + .collect::>() + .join(", ") ) } } @@ -229,6 +243,10 @@ impl RecordBatchStream for AvroStream<'_, R> { #[cfg(feature = "avro")] mod tests { + use crate::datasource::object_store::local::{ + local_sized_file, local_sized_file_stream, + }; + use super::*; #[tokio::test] @@ -240,10 +258,13 @@ mod tests { let testdata = crate::test_util::arrow_test_data(); let filename = format!("{}/avro/alltypes_plain.avro", testdata); let avro_exec = AvroExec::new( - vec![filename.clone()], + Arc::new(ObjectStoreRegistry::new()), + vec![PartitionedFile { + file: local_sized_file(filename.clone()), + }], Statistics::default(), - AvroFormat {} - .infer_schema(Box::pin(futures::stream::once(async { filename }))) + AvroFormat::default() + .infer_schema(local_sized_file_stream(vec![filename])) .await?, Some(vec![0, 1, 2]), 1024, diff --git a/datafusion/src/physical_plan/file_format/csv.rs b/datafusion/src/physical_plan/file_format/csv.rs index df3d74761564b..9989c3dd8bb14 100644 --- a/datafusion/src/physical_plan/file_format/csv.rs +++ b/datafusion/src/physical_plan/file_format/csv.rs @@ -17,6 +17,8 @@ //! Execution plan for reading CSV files +use crate::datasource::file_format::PartitionedFile; +use crate::datasource::object_store::{ObjectStoreRegistry, SizedFile}; use crate::error::{DataFusionError, Result}; use crate::physical_plan::{ DisplayFormatType, ExecutionPlan, Partitioning, RecordBatchStream, @@ -28,7 +30,6 @@ use arrow::error::Result as ArrowResult; use arrow::record_batch::RecordBatch; use futures::Stream; use std::any::Any; -use std::fs::File; use std::io::Read; use std::pin::Pin; use std::sync::Arc; @@ -39,8 +40,9 @@ use async_trait::async_trait; /// Execution plan for scanning a CSV file #[derive(Debug, Clone)] pub struct CsvExec { + object_store_registry: Arc, /// List of data files - files: Vec, + files: Vec, /// Schema representing the CSV file schema: SchemaRef, /// Provided statistics @@ -64,7 +66,8 @@ impl CsvExec { /// TODO: support partitiond file list (Vec>) #[allow(clippy::too_many_arguments)] pub fn new( - files: Vec, + object_store_registry: Arc, + files: Vec, statistics: Statistics, schema: SchemaRef, has_header: bool, @@ -81,6 +84,7 @@ impl CsvExec { }; Self { + object_store_registry, files, schema, statistics, @@ -132,7 +136,8 @@ impl ExecutionPlan for CsvExec { async fn execute(&self, partition: usize) -> Result { Ok(Box::pin(CsvStream::try_new( - &self.files[partition], + &self.object_store_registry, + &self.files[partition].file, self.schema.clone(), self.has_header, self.delimiter, @@ -151,11 +156,15 @@ impl ExecutionPlan for CsvExec { DisplayFormatType::Default => { write!( f, - "CsvExec: has_header={}, batch_size={}, limit={:?}, partitions=[{}]", + "CsvExec: has_header={}, batch_size={}, limit={:?}, files=[{}]", self.has_header, self.batch_size, self.limit, - self.files.join(", ") + self.files + .iter() + .map(|f| f.file.path.as_str()) + .collect::>() + .join(", ") ) } } @@ -171,10 +180,11 @@ struct CsvStream { /// Arrow CSV reader reader: csv::Reader, } -impl CsvStream { +impl CsvStream> { /// Create an iterator for a CSV file pub fn try_new( - filename: &str, + object_store_registry: &ObjectStoreRegistry, + file: &SizedFile, schema: SchemaRef, has_header: bool, delimiter: Option, @@ -182,12 +192,16 @@ impl CsvStream { batch_size: usize, limit: Option, ) -> Result { - let file = File::open(filename)?; + let file = object_store_registry + .get_by_uri(&file.path)? + .file_reader(file.clone())? + .chunk_reader(0, file.size as usize)?; Self::try_new_from_reader( file, schema, has_header, delimiter, projection, batch_size, limit, ) } } + impl CsvStream { /// Create an iterator for a reader pub fn try_new_from_reader( @@ -237,7 +251,9 @@ impl RecordBatchStream for CsvStream { #[cfg(test)] mod tests { use super::*; - use crate::test::aggr_test_schema; + use crate::{ + datasource::object_store::local::local_sized_file, test::aggr_test_schema, + }; use futures::StreamExt; #[tokio::test] @@ -247,7 +263,10 @@ mod tests { let filename = "aggregate_test_100.csv"; let path = format!("{}/csv/{}", testdata, filename); let csv = CsvExec::new( - vec![path], + Arc::new(ObjectStoreRegistry::new()), + vec![PartitionedFile { + file: local_sized_file(path), + }], Statistics::default(), schema, true, @@ -277,7 +296,10 @@ mod tests { let filename = "aggregate_test_100.csv"; let path = format!("{}/csv/{}", testdata, filename); let csv = CsvExec::new( - vec![path], + Arc::new(ObjectStoreRegistry::new()), + vec![PartitionedFile { + file: local_sized_file(path), + }], Statistics::default(), schema, true, diff --git a/datafusion/src/physical_plan/file_format/json.rs b/datafusion/src/physical_plan/file_format/json.rs index 69be9d2e7a9f9..923cd1bb5ecb5 100644 --- a/datafusion/src/physical_plan/file_format/json.rs +++ b/datafusion/src/physical_plan/file_format/json.rs @@ -19,6 +19,8 @@ use async_trait::async_trait; use futures::Stream; +use crate::datasource::file_format::PartitionedFile; +use crate::datasource::object_store::ObjectStoreRegistry; use crate::error::{DataFusionError, Result}; use crate::physical_plan::{ DisplayFormatType, ExecutionPlan, Partitioning, RecordBatchStream, @@ -31,7 +33,6 @@ use arrow::{ record_batch::RecordBatch, }; use std::any::Any; -use std::fs::File; use std::{ io::Read, pin::Pin, @@ -42,7 +43,8 @@ use std::{ /// Execution plan for scanning NdJson data source #[derive(Debug, Clone)] pub struct NdJsonExec { - files: Vec, + object_store_registry: Arc, + files: Vec, statistics: Statistics, schema: SchemaRef, projection: Option>, @@ -55,7 +57,8 @@ impl NdJsonExec { /// Create a new JSON reader execution plan provided file list and schema /// TODO: support partitiond file list (Vec>) pub fn new( - files: Vec, + object_store_registry: Arc, + files: Vec, statistics: Statistics, schema: SchemaRef, projection: Option>, @@ -70,6 +73,7 @@ impl NdJsonExec { }; Self { + object_store_registry, files, statistics, schema, @@ -114,24 +118,22 @@ impl ExecutionPlan for NdJsonExec { } async fn execute(&self, partition: usize) -> Result { - let mut builder = json::ReaderBuilder::new() - .with_schema(self.schema.clone()) - .with_batch_size(self.batch_size); - if let Some(proj) = &self.projection { - builder = builder.with_projection( - proj.iter() - .map(|col_idx| self.schema.field(*col_idx).name()) - .cloned() - .collect(), - ); - } - - let file = File::open(&self.files[partition])?; - - Ok(Box::pin(NdJsonStream::new( - builder.build(file)?, - self.limit, - ))) + let proj = self.projection.as_ref().map(|p| { + p.iter() + .map(|col_idx| self.schema.field(*col_idx).name()) + .cloned() + .collect() + }); + + let file = self + .object_store_registry + .get_by_uri(&self.files[partition].file.path)? + .file_reader(self.files[partition].file.clone())? + .chunk_reader(0, self.files[partition].file.size as usize)?; + + let json_reader = json::Reader::new(file, self.schema(), self.batch_size, proj); + + Ok(Box::pin(NdJsonStream::new(json_reader, self.limit))) } fn fmt_as( @@ -143,10 +145,14 @@ impl ExecutionPlan for NdJsonExec { DisplayFormatType::Default => { write!( f, - "JsonExec: batch_size={}, limit={:?}, partitions=[{}]", + "JsonExec: batch_size={}, limit={:?}, files=[{}]", self.batch_size, self.limit, - self.files.join(", ") + self.files + .iter() + .map(|f| f.file.path.as_str()) + .collect::>() + .join(", ") ) } } @@ -221,18 +227,19 @@ impl RecordBatchStream for NdJsonStream { mod tests { use futures::StreamExt; - use crate::datasource::file_format::{json::JsonFormat, FileFormat}; + use crate::datasource::{ + file_format::{json::JsonFormat, FileFormat}, + object_store::local::{local_sized_file, local_sized_file_stream}, + }; use super::*; const TEST_DATA_BASE: &str = "tests/jsons"; async fn infer_schema(path: String) -> Result { - JsonFormat { - schema_infer_max_rec: None, - } - .infer_schema(Box::pin(futures::stream::once(async { path }))) - .await + JsonFormat::default() + .infer_schema(local_sized_file_stream(vec![path])) + .await } #[tokio::test] @@ -240,7 +247,10 @@ mod tests { use arrow::datatypes::DataType; let path = format!("{}/1.json", TEST_DATA_BASE); let exec = NdJsonExec::new( - vec![path.clone()], + Arc::new(ObjectStoreRegistry::new()), + vec![PartitionedFile { + file: local_sized_file(path.clone()), + }], Default::default(), infer_schema(path).await?, None, @@ -292,7 +302,10 @@ mod tests { async fn nd_json_exec_file_projection() -> Result<()> { let path = format!("{}/1.json", TEST_DATA_BASE); let exec = NdJsonExec::new( - vec![path.clone()], + Arc::new(ObjectStoreRegistry::new()), + vec![PartitionedFile { + file: local_sized_file(path.clone()), + }], Default::default(), infer_schema(path).await?, Some(vec![0, 2]), diff --git a/datafusion/src/physical_plan/file_format/parquet.rs b/datafusion/src/physical_plan/file_format/parquet.rs index 615e52b544748..f745365c835a3 100644 --- a/datafusion/src/physical_plan/file_format/parquet.rs +++ b/datafusion/src/physical_plan/file_format/parquet.rs @@ -18,10 +18,11 @@ //! Execution plan for reading Parquet files use std::fmt; -use std::fs::File; use std::sync::Arc; use std::{any::Any, convert::TryInto}; +use crate::datasource::file_format::parquet::ChunkObjectReader; +use crate::datasource::object_store::ObjectStoreRegistry; use crate::{ error::{DataFusionError, Result}, logical_plan::{Column, Expr}, @@ -63,10 +64,11 @@ use crate::datasource::file_format::{FilePartition, PartitionedFile}; /// Execution plan for scanning one or more Parquet partitions #[derive(Debug, Clone)] pub struct ParquetExec { + object_store_registry: Arc, /// Parquet partitions to read - pub partitions: Vec, + partitions: Vec, /// Schema after projection is applied - pub schema: SchemaRef, + schema: SchemaRef, /// Projection for which columns to load projection: Vec, /// Batch size @@ -110,6 +112,7 @@ struct ParquetFileMetrics { impl ParquetExec { /// Create a new Parquet reader execution plan provided file list and schema pub fn new( + object_store_registry: Arc, files: Vec>, statistics: Statistics, schema: SchemaRef, @@ -156,6 +159,7 @@ impl ParquetExec { Self::project(&projection, schema, statistics); Self { + object_store_registry, partitions, schema: projected_schema, projection, @@ -283,9 +287,11 @@ impl ExecutionPlan for ParquetExec { let predicate_builder = self.predicate_builder.clone(); let batch_size = self.batch_size; let limit = self.limit; + let object_store_registry = Arc::clone(&self.object_store_registry); task::spawn_blocking(move || { if let Err(e) = read_partition( + &object_store_registry, partition_index, partition, metrics, @@ -462,6 +468,7 @@ fn build_row_group_predicate( #[allow(clippy::too_many_arguments)] fn read_partition( + object_store_registry: &ObjectStoreRegistry, partition_index: usize, partition: ParquetPartition, metrics: ExecutionPlanMetricsSet, @@ -474,10 +481,16 @@ fn read_partition( let mut total_rows = 0; let all_files = partition.file_partition.files; 'outer: for partitioned_file in all_files { - let file_metrics = - ParquetFileMetrics::new(partition_index, &*partitioned_file.path, &metrics); - let file = File::open(partitioned_file.path.as_str())?; - let mut file_reader = SerializedFileReader::new(file)?; + let file_metrics = ParquetFileMetrics::new( + partition_index, + &*partitioned_file.file.path, + &metrics, + ); + let object_reader = object_store_registry + .get_by_uri(&partitioned_file.file.path)? + .file_reader(partitioned_file.file.clone())?; + let mut file_reader = + SerializedFileReader::new(ChunkObjectReader(object_reader))?; if let Some(predicate_builder) = predicate_builder { let row_group_predicate = build_row_group_predicate( predicate_builder, @@ -526,7 +539,10 @@ fn read_partition( #[cfg(test)] mod tests { - use crate::datasource::file_format::{parquet::ParquetFormat, FileFormat}; + use crate::datasource::{ + file_format::{parquet::ParquetFormat, FileFormat}, + object_store::local::{local_sized_file, local_sized_file_stream}, + }; use super::*; use arrow::datatypes::{DataType, Field}; @@ -542,15 +558,14 @@ mod tests { let testdata = crate::test_util::parquet_test_data(); let filename = format!("{}/alltypes_plain.parquet", testdata); let parquet_exec = ParquetExec::new( + Arc::new(ObjectStoreRegistry::new()), vec![vec![PartitionedFile { - path: filename.clone(), + file: local_sized_file(filename.clone()), }]], Statistics::default(), - ParquetFormat { - enable_pruning: true, - } - .infer_schema(Box::pin(futures::stream::once(async { filename }))) - .await?, + ParquetFormat::default() + .infer_schema(local_sized_file_stream(vec![filename])) + .await?, Some(vec![0, 1, 2]), None, 1024, From 862d54bf7a0ff55599a15e038cd60ff4f614e8b8 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Mon, 4 Oct 2021 17:59:07 +0200 Subject: [PATCH 02/17] [fix] Clippy --- datafusion/src/datasource/file_format/avro.rs | 4 ++-- datafusion/src/datasource/listing.rs | 2 +- datafusion/src/physical_plan/file_format/csv.rs | 1 + datafusion/src/physical_plan/file_format/parquet.rs | 1 + 4 files changed, 5 insertions(+), 3 deletions(-) diff --git a/datafusion/src/datasource/file_format/avro.rs b/datafusion/src/datasource/file_format/avro.rs index c6f7b24f0e7db..dc3a253dbe462 100644 --- a/datafusion/src/datasource/file_format/avro.rs +++ b/datafusion/src/datasource/file_format/avro.rs @@ -382,7 +382,7 @@ mod tests { mod tests { use super::*; - use crate::datasource::file_format::string_stream; + use crate::datasource::object_store::local::local_sized_file_stream; use crate::error::DataFusionError; #[tokio::test] @@ -390,7 +390,7 @@ mod tests { let testdata = crate::test_util::arrow_test_data(); let filename = format!("{}/avro/alltypes_plain.avro", testdata); let schema_result = AvroFormat::default() - .infer_schema(string_stream(vec![filename])) + .infer_schema(local_sized_file_stream(vec![filename])) .await; assert!(matches!( schema_result, diff --git a/datafusion/src/datasource/listing.rs b/datafusion/src/datasource/listing.rs index 3798aaff46868..5c7b46929026f 100644 --- a/datafusion/src/datasource/listing.rs +++ b/datafusion/src/datasource/listing.rs @@ -128,7 +128,7 @@ impl TableProvider for ListingTable { ) -> Result> { // list files (with partitions) let file_list = pruned_partition_list( - &self.options.format.object_store_registry(), + self.options.format.object_store_registry(), &self.path, filters, &self.options.file_extension, diff --git a/datafusion/src/physical_plan/file_format/csv.rs b/datafusion/src/physical_plan/file_format/csv.rs index 9989c3dd8bb14..ca8f922b1e9fa 100644 --- a/datafusion/src/physical_plan/file_format/csv.rs +++ b/datafusion/src/physical_plan/file_format/csv.rs @@ -182,6 +182,7 @@ struct CsvStream { } impl CsvStream> { /// Create an iterator for a CSV file + #[allow(clippy::too_many_arguments)] pub fn try_new( object_store_registry: &ObjectStoreRegistry, file: &SizedFile, diff --git a/datafusion/src/physical_plan/file_format/parquet.rs b/datafusion/src/physical_plan/file_format/parquet.rs index f745365c835a3..0ae1b073a22c1 100644 --- a/datafusion/src/physical_plan/file_format/parquet.rs +++ b/datafusion/src/physical_plan/file_format/parquet.rs @@ -111,6 +111,7 @@ struct ParquetFileMetrics { impl ParquetExec { /// Create a new Parquet reader execution plan provided file list and schema + #[allow(clippy::too_many_arguments)] pub fn new( object_store_registry: Arc, files: Vec>, From 3f0c146f6df5ccc52e4f39c2778517f50be5f332 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Mon, 4 Oct 2021 17:59:07 +0200 Subject: [PATCH 03/17] [test] improve file_format tests with limit --- datafusion/src/datasource/file_format/avro.rs | 43 +++++++++++---- datafusion/src/datasource/file_format/csv.rs | 33 +++++++++--- datafusion/src/datasource/file_format/json.rs | 33 +++++++++--- .../src/datasource/file_format/parquet.rs | 52 ++++++++++++++----- .../src/physical_plan/file_format/parquet.rs | 3 +- 5 files changed, 128 insertions(+), 36 deletions(-) diff --git a/datafusion/src/datasource/file_format/avro.rs b/datafusion/src/datasource/file_format/avro.rs index dc3a253dbe462..cd2f94cea66b7 100644 --- a/datafusion/src/datasource/file_format/avro.rs +++ b/datafusion/src/datasource/file_format/avro.rs @@ -124,10 +124,10 @@ mod tests { #[tokio::test] async fn read_small_batches() -> Result<()> { let projection = None; - let exec = get_exec("alltypes_plain.avro", &projection, 2).await?; + let exec = get_exec("alltypes_plain.avro", &projection, 2, None).await?; let stream = exec.execute(0).await?; - let _ = stream + let tt_batches = stream .map(|batch| { let batch = batch.unwrap(); assert_eq!(11, batch.num_columns()); @@ -136,13 +136,27 @@ mod tests { .fold(0, |acc, _| async move { acc + 1i32 }) .await; + assert_eq!(tt_batches, 4 /* 8/2 */); + + Ok(()) + } + + #[tokio::test] + async fn read_limit() -> Result<()> { + let projection = None; + let exec = get_exec("alltypes_plain.avro", &projection, 1024, Some(1)).await?; + let batches = collect(exec).await?; + assert_eq!(1, batches.len()); + assert_eq!(11, batches[0].num_columns()); + assert_eq!(1, batches[0].num_rows()); + Ok(()) } #[tokio::test] async fn read_alltypes_plain_avro() -> Result<()> { let projection = None; - let exec = get_exec("alltypes_plain.avro", &projection, 1024).await?; + let exec = get_exec("alltypes_plain.avro", &projection, 1024, None).await?; let x: Vec = exec .schema() @@ -192,7 +206,7 @@ mod tests { #[tokio::test] async fn read_bool_alltypes_plain_avro() -> Result<()> { let projection = Some(vec![1]); - let exec = get_exec("alltypes_plain.avro", &projection, 1024).await?; + let exec = get_exec("alltypes_plain.avro", &projection, 1024, None).await?; let batches = collect(exec).await?; assert_eq!(batches.len(), 1); @@ -220,7 +234,7 @@ mod tests { #[tokio::test] async fn read_i32_alltypes_plain_avro() -> Result<()> { let projection = Some(vec![0]); - let exec = get_exec("alltypes_plain.avro", &projection, 1024).await?; + let exec = get_exec("alltypes_plain.avro", &projection, 1024, None).await?; let batches = collect(exec).await?; assert_eq!(batches.len(), 1); @@ -245,7 +259,7 @@ mod tests { #[tokio::test] async fn read_i96_alltypes_plain_avro() -> Result<()> { let projection = Some(vec![10]); - let exec = get_exec("alltypes_plain.avro", &projection, 1024).await?; + let exec = get_exec("alltypes_plain.avro", &projection, 1024, None).await?; let batches = collect(exec).await?; assert_eq!(batches.len(), 1); @@ -270,7 +284,7 @@ mod tests { #[tokio::test] async fn read_f32_alltypes_plain_avro() -> Result<()> { let projection = Some(vec![6]); - let exec = get_exec("alltypes_plain.avro", &projection, 1024).await?; + let exec = get_exec("alltypes_plain.avro", &projection, 1024, None).await?; let batches = collect(exec).await?; assert_eq!(batches.len(), 1); @@ -298,7 +312,7 @@ mod tests { #[tokio::test] async fn read_f64_alltypes_plain_avro() -> Result<()> { let projection = Some(vec![7]); - let exec = get_exec("alltypes_plain.avro", &projection, 1024).await?; + let exec = get_exec("alltypes_plain.avro", &projection, 1024, None).await?; let batches = collect(exec).await?; assert_eq!(batches.len(), 1); @@ -326,7 +340,7 @@ mod tests { #[tokio::test] async fn read_binary_alltypes_plain_avro() -> Result<()> { let projection = Some(vec![9]); - let exec = get_exec("alltypes_plain.avro", &projection, 1024).await?; + let exec = get_exec("alltypes_plain.avro", &projection, 1024, None).await?; let batches = collect(exec).await?; assert_eq!(batches.len(), 1); @@ -355,6 +369,7 @@ mod tests { file_name: &str, projection: &Option>, batch_size: usize, + limit: Option, ) -> Result> { let testdata = crate::test_util::arrow_test_data(); let filename = format!("{}/avro/{}", testdata, file_name); @@ -371,7 +386,15 @@ mod tests { file: local_sized_file(filename.to_owned()), }]]; let exec = format - .create_physical_plan(schema, files, stats, projection, batch_size, &[], None) + .create_physical_plan( + schema, + files, + stats, + projection, + batch_size, + &[], + limit, + ) .await?; Ok(exec) } diff --git a/datafusion/src/datasource/file_format/csv.rs b/datafusion/src/datasource/file_format/csv.rs index 288c9c6faa6fa..d8ce6282bdf03 100644 --- a/datafusion/src/datasource/file_format/csv.rs +++ b/datafusion/src/datasource/file_format/csv.rs @@ -164,10 +164,10 @@ mod tests { async fn read_small_batches() -> Result<()> { // skip column 9 that overflows the automaticly discovered column type of i64 (u64 would work) let projection = Some(vec![0, 1, 2, 3, 4, 5, 6, 7, 8, 10, 11, 12]); - let exec = get_exec("aggregate_test_100.csv", &projection, 2).await?; + let exec = get_exec("aggregate_test_100.csv", &projection, 2, None).await?; let stream = exec.execute(0).await?; - let tt_rows: i32 = stream + let tt_batches: i32 = stream .map(|batch| { let batch = batch.unwrap(); assert_eq!(12, batch.num_columns()); @@ -176,7 +176,7 @@ mod tests { .fold(0, |acc, _| async move { acc + 1i32 }) .await; - assert_eq!(tt_rows, 50 /* 100/2 */); + assert_eq!(tt_batches, 50 /* 100/2 */); // test metadata assert_eq!(exec.statistics().num_rows, None); @@ -185,10 +185,22 @@ mod tests { Ok(()) } + #[tokio::test] + async fn read_limit() -> Result<()> { + let projection = Some(vec![0, 1, 2, 3]); + let exec = get_exec("aggregate_test_100.csv", &projection, 1024, Some(1)).await?; + let batches = collect(exec).await?; + assert_eq!(1, batches.len()); + assert_eq!(4, batches[0].num_columns()); + assert_eq!(1, batches[0].num_rows()); + + Ok(()) + } + #[tokio::test] async fn infer_schema() -> Result<()> { let projection = None; - let exec = get_exec("aggregate_test_100.csv", &projection, 1024).await?; + let exec = get_exec("aggregate_test_100.csv", &projection, 1024, None).await?; let x: Vec = exec .schema() @@ -221,7 +233,7 @@ mod tests { #[tokio::test] async fn read_char_column() -> Result<()> { let projection = Some(vec![0]); - let exec = get_exec("aggregate_test_100.csv", &projection, 1024).await?; + let exec = get_exec("aggregate_test_100.csv", &projection, 1024, None).await?; let batches = collect(exec).await.expect("Collect batches"); @@ -248,6 +260,7 @@ mod tests { file_name: &str, projection: &Option>, batch_size: usize, + limit: Option, ) -> Result> { let testdata = crate::test_util::arrow_test_data(); let filename = format!("{}/csv/{}", testdata, file_name); @@ -264,7 +277,15 @@ mod tests { file: local_sized_file(filename.to_owned()), }]]; let exec = format - .create_physical_plan(schema, files, stats, projection, batch_size, &[], None) + .create_physical_plan( + schema, + files, + stats, + projection, + batch_size, + &[], + limit, + ) .await?; Ok(exec) } diff --git a/datafusion/src/datasource/file_format/json.rs b/datafusion/src/datasource/file_format/json.rs index 02f5fcd5a9557..5512497e32893 100644 --- a/datafusion/src/datasource/file_format/json.rs +++ b/datafusion/src/datasource/file_format/json.rs @@ -144,10 +144,10 @@ mod tests { #[tokio::test] async fn read_small_batches() -> Result<()> { let projection = None; - let exec = get_exec(&projection, 2).await?; + let exec = get_exec(&projection, 2, None).await?; let stream = exec.execute(0).await?; - let tt_rows: i32 = stream + let tt_batches: i32 = stream .map(|batch| { let batch = batch.unwrap(); assert_eq!(4, batch.num_columns()); @@ -156,7 +156,7 @@ mod tests { .fold(0, |acc, _| async move { acc + 1i32 }) .await; - assert_eq!(tt_rows, 6 /* 12/2 */); + assert_eq!(tt_batches, 6 /* 12/2 */); // test metadata assert_eq!(exec.statistics().num_rows, None); @@ -165,10 +165,22 @@ mod tests { Ok(()) } + #[tokio::test] + async fn read_limit() -> Result<()> { + let projection = None; + let exec = get_exec(&projection, 1024, Some(1)).await?; + let batches = collect(exec).await?; + assert_eq!(1, batches.len()); + assert_eq!(4, batches[0].num_columns()); + assert_eq!(1, batches[0].num_rows()); + + Ok(()) + } + #[tokio::test] async fn infer_schema() -> Result<()> { let projection = None; - let exec = get_exec(&projection, 1024).await?; + let exec = get_exec(&projection, 1024, None).await?; let x: Vec = exec .schema() @@ -184,7 +196,7 @@ mod tests { #[tokio::test] async fn read_int_column() -> Result<()> { let projection = Some(vec![0]); - let exec = get_exec(&projection, 1024).await?; + let exec = get_exec(&projection, 1024, None).await?; let batches = collect(exec).await.expect("Collect batches"); @@ -213,6 +225,7 @@ mod tests { async fn get_exec( projection: &Option>, batch_size: usize, + limit: Option, ) -> Result> { let filename = "tests/jsons/2.json"; let format = JsonFormat::default(); @@ -228,7 +241,15 @@ mod tests { file: local_sized_file(filename.to_owned()), }]]; let exec = format - .create_physical_plan(schema, files, stats, projection, batch_size, &[], None) + .create_physical_plan( + schema, + files, + stats, + projection, + batch_size, + &[], + limit, + ) .await?; Ok(exec) } diff --git a/datafusion/src/datasource/file_format/parquet.rs b/datafusion/src/datasource/file_format/parquet.rs index 26b49d76cc97d..99a8a890914f2 100644 --- a/datafusion/src/datasource/file_format/parquet.rs +++ b/datafusion/src/datasource/file_format/parquet.rs @@ -128,9 +128,7 @@ impl FileFormat for ParquetFormat { schema, projection.clone(), predicate, - limit - .map(|l| std::cmp::min(l, batch_size)) - .unwrap_or(batch_size), + batch_size, limit, ))) } @@ -361,10 +359,10 @@ mod tests { #[tokio::test] async fn read_small_batches() -> Result<()> { let projection = None; - let exec = get_exec("alltypes_plain.parquet", &projection, 2).await?; + let exec = get_exec("alltypes_plain.parquet", &projection, 2, None).await?; let stream = exec.execute(0).await?; - let _ = stream + let tt_batches = stream .map(|batch| { let batch = batch.unwrap(); assert_eq!(11, batch.num_columns()); @@ -373,6 +371,8 @@ mod tests { .fold(0, |acc, _| async move { acc + 1i32 }) .await; + assert_eq!(tt_batches, 4 /* 8/2 */); + // test metadata assert_eq!(exec.statistics().num_rows, Some(8)); assert_eq!(exec.statistics().total_byte_size, Some(671)); @@ -380,10 +380,27 @@ mod tests { Ok(()) } + #[tokio::test] + async fn read_limit() -> Result<()> { + let projection = None; + let exec = get_exec("alltypes_plain.parquet", &projection, 1024, Some(1)).await?; + + // note: even if the limit is set, the executor rounds up to the batch size + assert_eq!(exec.statistics().num_rows, Some(8)); + assert_eq!(exec.statistics().total_byte_size, Some(671)); + assert!(exec.statistics().is_exact); + let batches = collect(exec).await?; + assert_eq!(1, batches.len()); + assert_eq!(11, batches[0].num_columns()); + assert_eq!(8, batches[0].num_rows()); + + Ok(()) + } + #[tokio::test] async fn read_alltypes_plain_parquet() -> Result<()> { let projection = None; - let exec = get_exec("alltypes_plain.parquet", &projection, 1024).await?; + let exec = get_exec("alltypes_plain.parquet", &projection, 1024, None).await?; let x: Vec = exec .schema() @@ -419,7 +436,7 @@ mod tests { #[tokio::test] async fn read_bool_alltypes_plain_parquet() -> Result<()> { let projection = Some(vec![1]); - let exec = get_exec("alltypes_plain.parquet", &projection, 1024).await?; + let exec = get_exec("alltypes_plain.parquet", &projection, 1024, None).await?; let batches = collect(exec).await?; assert_eq!(1, batches.len()); @@ -447,7 +464,7 @@ mod tests { #[tokio::test] async fn read_i32_alltypes_plain_parquet() -> Result<()> { let projection = Some(vec![0]); - let exec = get_exec("alltypes_plain.parquet", &projection, 1024).await?; + let exec = get_exec("alltypes_plain.parquet", &projection, 1024, None).await?; let batches = collect(exec).await?; assert_eq!(1, batches.len()); @@ -472,7 +489,7 @@ mod tests { #[tokio::test] async fn read_i96_alltypes_plain_parquet() -> Result<()> { let projection = Some(vec![10]); - let exec = get_exec("alltypes_plain.parquet", &projection, 1024).await?; + let exec = get_exec("alltypes_plain.parquet", &projection, 1024, None).await?; let batches = collect(exec).await?; assert_eq!(1, batches.len()); @@ -497,7 +514,7 @@ mod tests { #[tokio::test] async fn read_f32_alltypes_plain_parquet() -> Result<()> { let projection = Some(vec![6]); - let exec = get_exec("alltypes_plain.parquet", &projection, 1024).await?; + let exec = get_exec("alltypes_plain.parquet", &projection, 1024, None).await?; let batches = collect(exec).await?; assert_eq!(1, batches.len()); @@ -525,7 +542,7 @@ mod tests { #[tokio::test] async fn read_f64_alltypes_plain_parquet() -> Result<()> { let projection = Some(vec![7]); - let exec = get_exec("alltypes_plain.parquet", &projection, 1024).await?; + let exec = get_exec("alltypes_plain.parquet", &projection, 1024, None).await?; let batches = collect(exec).await?; assert_eq!(1, batches.len()); @@ -553,7 +570,7 @@ mod tests { #[tokio::test] async fn read_binary_alltypes_plain_parquet() -> Result<()> { let projection = Some(vec![9]); - let exec = get_exec("alltypes_plain.parquet", &projection, 1024).await?; + let exec = get_exec("alltypes_plain.parquet", &projection, 1024, None).await?; let batches = collect(exec).await?; assert_eq!(1, batches.len()); @@ -582,6 +599,7 @@ mod tests { file_name: &str, projection: &Option>, batch_size: usize, + limit: Option, ) -> Result> { let testdata = crate::test_util::parquet_test_data(); let filename = format!("{}/{}", testdata, file_name); @@ -598,7 +616,15 @@ mod tests { file: local_sized_file(filename.clone()), }]]; let exec = format - .create_physical_plan(schema, files, stats, projection, batch_size, &[], None) + .create_physical_plan( + schema, + files, + stats, + projection, + batch_size, + &[], + limit, + ) .await?; Ok(exec) } diff --git a/datafusion/src/physical_plan/file_format/parquet.rs b/datafusion/src/physical_plan/file_format/parquet.rs index 0ae1b073a22c1..433182abb7bf6 100644 --- a/datafusion/src/physical_plan/file_format/parquet.rs +++ b/datafusion/src/physical_plan/file_format/parquet.rs @@ -110,7 +110,8 @@ struct ParquetFileMetrics { } impl ParquetExec { - /// Create a new Parquet reader execution plan provided file list and schema + /// Create a new Parquet reader execution plan provided file list and schema. + /// Even if `limit` is set, ParquetExec rounds up the number of records to the next `batch_size`. #[allow(clippy::too_many_arguments)] pub fn new( object_store_registry: Arc, From b7c3a388b67e6a78f3e03a58052b5ed0c7e0c764 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Mon, 4 Oct 2021 17:59:07 +0200 Subject: [PATCH 04/17] [fix] limit file system read size --- datafusion/src/datasource/object_store/local.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/src/datasource/object_store/local.rs b/datafusion/src/datasource/object_store/local.rs index 18e74e2e11618..a99616f08ec5d 100644 --- a/datafusion/src/datasource/object_store/local.rs +++ b/datafusion/src/datasource/object_store/local.rs @@ -68,11 +68,11 @@ impl ObjectReader for LocalFileReader { fn chunk_reader( &self, start: u64, - _length: usize, + length: usize, ) -> Result> { let mut file = File::open(&self.file.path)?; file.seek(SeekFrom::Start(start))?; - Ok(Box::new(file)) + Ok(Box::new(file.take(length as u64))) } fn length(&self) -> u64 { From 89fa9bba11e3895e5d14f1e9f107f52c8a73627f Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Mon, 4 Oct 2021 17:59:07 +0200 Subject: [PATCH 05/17] [fix] avoid fetching unnecessary stats after limit --- datafusion/src/datasource/file_format/mod.rs | 27 ++++++++-------- datafusion/src/datasource/listing.rs | 31 +++++++++---------- .../src/datasource/object_store/local.rs | 2 +- datafusion/src/datasource/object_store/mod.rs | 12 +++---- 4 files changed, 35 insertions(+), 37 deletions(-) diff --git a/datafusion/src/datasource/file_format/mod.rs b/datafusion/src/datasource/file_format/mod.rs index 8ce30a5a9a151..b6e5f8a3f2777 100644 --- a/datafusion/src/datasource/file_format/mod.rs +++ b/datafusion/src/datasource/file_format/mod.rs @@ -32,7 +32,7 @@ use crate::logical_plan::Expr; use crate::physical_plan::{Accumulator, ExecutionPlan, Statistics}; use async_trait::async_trait; -use futures::Stream; +use futures::{Stream, StreamExt}; use super::object_store::{ObjectStoreRegistry, SizedFile, SizedFileStream}; @@ -81,12 +81,12 @@ pub trait FileFormat: Send + Sync { /// needed to read up to `limit` number of rows /// TODO fix case where `num_rows` and `total_byte_size` are not defined (stat should be None instead of Some(0)) /// TODO move back to crate::datasource::mod.rs once legacy cleaned up -pub fn get_statistics_with_limit( - all_files: &[(PartitionedFile, Statistics)], +pub async fn get_statistics_with_limit( + all_files: impl Stream>, schema: SchemaRef, limit: Option, -) -> (Vec, Statistics) { - let mut all_files = all_files.to_vec(); +) -> Result<(Vec, Statistics)> { + let mut result_files = vec![]; let mut total_byte_size = 0; let mut null_counts = vec![0; schema.fields().len()]; @@ -94,10 +94,11 @@ pub fn get_statistics_with_limit( let (mut max_values, mut min_values) = create_max_min_accs(&schema); let mut num_rows = 0; - let mut num_files = 0; let mut is_exact = true; - for (_, file_stats) in &all_files { - num_files += 1; + let mut all_files = Box::pin(all_files); + while let Some(res) = all_files.next().await { + let (file, file_stats) = res?; + result_files.push(file); is_exact &= file_stats.is_exact; num_rows += file_stats.num_rows.unwrap_or(0); total_byte_size += file_stats.total_byte_size.unwrap_or(0); @@ -133,9 +134,11 @@ pub fn get_statistics_with_limit( break; } } - if num_files < all_files.len() { + // if we still have files in the stream, it means that the limit kicked + // in and that the statistic could have been different if we processed + // the files in a different order. + if all_files.next().await.is_some() { is_exact = false; - all_files.truncate(num_files); } let column_stats = if has_statistics { @@ -156,9 +159,7 @@ pub fn get_statistics_with_limit( is_exact, }; - let files = all_files.into_iter().map(|(f, _)| f).collect(); - - (files, statistics) + Ok((result_files, statistics)) } #[derive(Debug, Clone)] diff --git a/datafusion/src/datasource/listing.rs b/datafusion/src/datasource/listing.rs index 5c7b46929026f..bd5ba6804dc7c 100644 --- a/datafusion/src/datasource/listing.rs +++ b/datafusion/src/datasource/listing.rs @@ -22,7 +22,7 @@ use std::{any::Any, sync::Arc}; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use async_trait::async_trait; -use futures::{StreamExt, TryStreamExt}; +use futures::StreamExt; use crate::{ datasource::file_format::{self, PartitionedFile}, @@ -137,24 +137,21 @@ impl TableProvider for ListingTable { .await?; // collect the statistics if required by the config - let files = file_list - .then(|part_file| async { - let part_file = part_file?; - let statistics = if self.options.collect_stat { - self.options - .format - .infer_stats(part_file.file.clone()) - .await? - } else { - Statistics::default() - }; - Ok((part_file, statistics)) as Result<(PartitionedFile, Statistics)> - }) - .try_collect::>() - .await?; + let files = file_list.then(|part_file| async { + let part_file = part_file?; + let statistics = if self.options.collect_stat { + self.options + .format + .infer_stats(part_file.file.clone()) + .await? + } else { + Statistics::default() + }; + Ok((part_file, statistics)) as Result<(PartitionedFile, Statistics)> + }); let (files, statistics) = - file_format::get_statistics_with_limit(&files, self.schema(), limit); + file_format::get_statistics_with_limit(files, self.schema(), limit).await?; let partitioned_file_lists = split_files(files, self.options.max_partitions); diff --git a/datafusion/src/datasource/object_store/local.rs b/datafusion/src/datasource/object_store/local.rs index a99616f08ec5d..a68de29e003fe 100644 --- a/datafusion/src/datasource/object_store/local.rs +++ b/datafusion/src/datasource/object_store/local.rs @@ -135,7 +135,7 @@ async fn list_all(prefix: String) -> Result { } } -/// Create a stream of `SizedFile` applying `local_sized_file` to each path +/// Create a stream of `SizedFile` applying `local_sized_file` to each path in `files` pub fn local_sized_file_stream(files: Vec) -> SizedFileStream { Box::pin(futures::stream::iter(files).map(|f| Ok(local_sized_file(f)))) } diff --git a/datafusion/src/datasource/object_store/mod.rs b/datafusion/src/datasource/object_store/mod.rs index 08bbad020eb5c..ab4735ccacf52 100644 --- a/datafusion/src/datasource/object_store/mod.rs +++ b/datafusion/src/datasource/object_store/mod.rs @@ -37,27 +37,27 @@ Object Reader for one file in an object store */ #[async_trait] pub trait ObjectReader { - /// Get reader for a part [start, start + length] in the file asynchronously + /// Get reader for a part [start, start + length] in the file fn chunk_reader( &self, start: u64, length: usize, ) -> Result>; - /// Get length for the file + /// Get the size of the file fn length(&self) -> u64; } /// Represents a file or a prefix that may require further resolution #[derive(Debug)] pub enum ListEntry { - /// File metadata + /// Complete file path with size SizedFile(SizedFile), /// Prefix to be further resolved during partition discovery Prefix(String), } -/// File meta we got from object store +/// Complete file path with size we got from object store #[derive(Debug, Clone)] pub struct SizedFile { /// Path of the file @@ -72,11 +72,11 @@ impl std::fmt::Display for SizedFile { } } -/// Stream of files get listed from object store +/// Stream of files listed from object store pub type SizedFileStream = Pin> + Send + Sync + 'static>>; -/// Stream of list entries get from object store +/// Stream of list entries obtained from object store pub type ListEntryStream = Pin> + Send + Sync + 'static>>; From 5d3170d32f80b1ecfa24a0907e450d804f3e6dae Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Mon, 4 Oct 2021 17:59:07 +0200 Subject: [PATCH 06/17] [fix] improve readability --- datafusion/src/datasource/file_format/avro.rs | 11 ++++++----- datafusion/src/datasource/file_format/csv.rs | 10 +++++----- datafusion/src/datasource/file_format/json.rs | 12 ++++++------ datafusion/src/datasource/object_store/mod.rs | 5 +++++ datafusion/src/physical_plan/file_format/avro.rs | 2 +- datafusion/src/physical_plan/file_format/csv.rs | 2 +- datafusion/src/physical_plan/file_format/json.rs | 2 +- 7 files changed, 25 insertions(+), 19 deletions(-) diff --git a/datafusion/src/datasource/file_format/avro.rs b/datafusion/src/datasource/file_format/avro.rs index cd2f94cea66b7..0e6008799271e 100644 --- a/datafusion/src/datasource/file_format/avro.rs +++ b/datafusion/src/datasource/file_format/avro.rs @@ -34,7 +34,7 @@ use crate::physical_plan::file_format::AvroExec; use crate::physical_plan::ExecutionPlan; use crate::physical_plan::Statistics; -/// Line-delimited Avro `FileFormat` implementation. +/// Avro `FileFormat` implementation. pub struct AvroFormat { /// Object store registry pub object_store_registry: Arc, @@ -63,10 +63,11 @@ impl FileFormat for AvroFormat { let mut schemas = vec![]; while let Some(fmeta_res) = file_stream.next().await { let fmeta = fmeta_res?; - let fsize = fmeta.size as usize; - let object_store = self.object_store_registry.get_by_uri(&fmeta.path)?; - let obj_reader = object_store.file_reader(fmeta)?; - let mut reader = obj_reader.chunk_reader(0, fsize)?; + let mut reader = self + .object_store_registry + .get_by_uri(&fmeta.path)? + .file_reader(fmeta)? + .reader()?; let schema = read_avro_schema_from_reader(&mut reader)?; schemas.push(schema); } diff --git a/datafusion/src/datasource/file_format/csv.rs b/datafusion/src/datasource/file_format/csv.rs index d8ce6282bdf03..9f8b27cb9114d 100644 --- a/datafusion/src/datasource/file_format/csv.rs +++ b/datafusion/src/datasource/file_format/csv.rs @@ -91,11 +91,11 @@ impl FileFormat for CsvFormat { while let Some(fmeta_res) = file_stream.next().await { let fmeta = fmeta_res?; - let fsize = fmeta.size as usize; - let object_store = self.object_store_registry.get_by_uri(&fmeta.path)?; - - let obj_reader = object_store.file_reader(fmeta)?; - let mut reader = obj_reader.chunk_reader(0, fsize)?; + let mut reader = self + .object_store_registry + .get_by_uri(&fmeta.path)? + .file_reader(fmeta)? + .reader()?; let (schema, records_read) = arrow::csv::reader::infer_reader_schema( &mut reader, self.delimiter, diff --git a/datafusion/src/datasource/file_format/json.rs b/datafusion/src/datasource/file_format/json.rs index 5512497e32893..20441b2d62c26 100644 --- a/datafusion/src/datasource/file_format/json.rs +++ b/datafusion/src/datasource/file_format/json.rs @@ -77,12 +77,12 @@ impl FileFormat for JsonFormat { let mut records_to_read = self.schema_infer_max_rec.unwrap_or(usize::MAX); while let Some(fmeta_res) = file_stream.next().await { let fmeta = fmeta_res?; - let fsize = fmeta.size as usize; - let object_store = self.object_store_registry.get_by_uri(&fmeta.path)?; - - let obj_reader = object_store.file_reader(fmeta)?; - let chunk_reader = obj_reader.chunk_reader(0, fsize)?; - let mut reader = BufReader::new(chunk_reader); + let reader = self + .object_store_registry + .get_by_uri(&fmeta.path)? + .file_reader(fmeta)? + .reader()?; + let mut reader = BufReader::new(reader); let iter = ValueIter::new(&mut reader, None); let schema = infer_json_schema_from_iterator(iter.take_while(|_| { let should_take = records_to_read > 0; diff --git a/datafusion/src/datasource/object_store/mod.rs b/datafusion/src/datasource/object_store/mod.rs index ab4735ccacf52..a63f55e447d41 100644 --- a/datafusion/src/datasource/object_store/mod.rs +++ b/datafusion/src/datasource/object_store/mod.rs @@ -44,6 +44,11 @@ pub trait ObjectReader { length: usize, ) -> Result>; + /// Get reader for the entire file + fn reader(&self) -> Result> { + self.chunk_reader(0, self.length() as usize) + } + /// Get the size of the file fn length(&self) -> u64; } diff --git a/datafusion/src/physical_plan/file_format/avro.rs b/datafusion/src/physical_plan/file_format/avro.rs index 5259780e16ea2..e20ba3afe7a86 100644 --- a/datafusion/src/physical_plan/file_format/avro.rs +++ b/datafusion/src/physical_plan/file_format/avro.rs @@ -129,7 +129,7 @@ impl ExecutionPlan for AvroExec { .object_store_registry .get_by_uri(&self.files[partition].file.path)? .file_reader(self.files[partition].file.clone())? - .chunk_reader(0, self.files[partition].file.size as usize)?; + .reader()?; let proj = self.projection.as_ref().map(|p| { p.iter() diff --git a/datafusion/src/physical_plan/file_format/csv.rs b/datafusion/src/physical_plan/file_format/csv.rs index ca8f922b1e9fa..d7392579774ca 100644 --- a/datafusion/src/physical_plan/file_format/csv.rs +++ b/datafusion/src/physical_plan/file_format/csv.rs @@ -196,7 +196,7 @@ impl CsvStream> { let file = object_store_registry .get_by_uri(&file.path)? .file_reader(file.clone())? - .chunk_reader(0, file.size as usize)?; + .reader()?; Self::try_new_from_reader( file, schema, has_header, delimiter, projection, batch_size, limit, ) diff --git a/datafusion/src/physical_plan/file_format/json.rs b/datafusion/src/physical_plan/file_format/json.rs index 923cd1bb5ecb5..1d02f5fca3b24 100644 --- a/datafusion/src/physical_plan/file_format/json.rs +++ b/datafusion/src/physical_plan/file_format/json.rs @@ -129,7 +129,7 @@ impl ExecutionPlan for NdJsonExec { .object_store_registry .get_by_uri(&self.files[partition].file.path)? .file_reader(self.files[partition].file.clone())? - .chunk_reader(0, self.files[partition].file.size as usize)?; + .reader()?; let json_reader = json::Reader::new(file, self.schema(), self.batch_size, proj); From d108701c29aad79a79189e623de54d195818d427 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Mon, 4 Oct 2021 17:59:07 +0200 Subject: [PATCH 07/17] [doc] improve comments --- datafusion/src/datasource/object_store/local.rs | 2 ++ datafusion/src/datasource/object_store/mod.rs | 6 +++--- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/datafusion/src/datasource/object_store/local.rs b/datafusion/src/datasource/object_store/local.rs index a68de29e003fe..8fe46fad37aed 100644 --- a/datafusion/src/datasource/object_store/local.rs +++ b/datafusion/src/datasource/object_store/local.rs @@ -70,6 +70,8 @@ impl ObjectReader for LocalFileReader { start: u64, length: usize, ) -> Result> { + // A new file descriptor is opened for each chunk reader. + // This okay because chunks are usually fairly large. let mut file = File::open(&self.file.path)?; file.seek(SeekFrom::Start(start))?; Ok(Box::new(file.take(length as u64))) diff --git a/datafusion/src/datasource/object_store/mod.rs b/datafusion/src/datasource/object_store/mod.rs index a63f55e447d41..b03c795024b22 100644 --- a/datafusion/src/datasource/object_store/mod.rs +++ b/datafusion/src/datasource/object_store/mod.rs @@ -32,9 +32,9 @@ use local::LocalFileSystem; use crate::error::{DataFusionError, Result}; -/** -Object Reader for one file in an object store -*/ +/// Object Reader for one file in an object store +/// Note that the dynamic dispatch on the reader might +/// have some performance impacts. #[async_trait] pub trait ObjectReader { /// Get reader for a part [start, start + length] in the file From 08f219dacc62da83148ecdca1b3f90aacb531c43 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Mon, 4 Oct 2021 17:59:07 +0200 Subject: [PATCH 08/17] [refacto] keep async reader stub --- datafusion/src/datasource/file_format/avro.rs | 2 +- datafusion/src/datasource/file_format/csv.rs | 2 +- datafusion/src/datasource/file_format/json.rs | 2 +- datafusion/src/datasource/file_format/parquet.rs | 2 +- datafusion/src/datasource/object_store/local.rs | 14 ++++++++++++-- datafusion/src/datasource/object_store/mod.rs | 12 ++++++++---- datafusion/src/physical_plan/file_format/avro.rs | 2 +- datafusion/src/physical_plan/file_format/csv.rs | 2 +- datafusion/src/physical_plan/file_format/json.rs | 2 +- 9 files changed, 27 insertions(+), 13 deletions(-) diff --git a/datafusion/src/datasource/file_format/avro.rs b/datafusion/src/datasource/file_format/avro.rs index 0e6008799271e..ac83d63296ae1 100644 --- a/datafusion/src/datasource/file_format/avro.rs +++ b/datafusion/src/datasource/file_format/avro.rs @@ -67,7 +67,7 @@ impl FileFormat for AvroFormat { .object_store_registry .get_by_uri(&fmeta.path)? .file_reader(fmeta)? - .reader()?; + .sync_reader()?; let schema = read_avro_schema_from_reader(&mut reader)?; schemas.push(schema); } diff --git a/datafusion/src/datasource/file_format/csv.rs b/datafusion/src/datasource/file_format/csv.rs index 9f8b27cb9114d..302afb4c0d9c3 100644 --- a/datafusion/src/datasource/file_format/csv.rs +++ b/datafusion/src/datasource/file_format/csv.rs @@ -95,7 +95,7 @@ impl FileFormat for CsvFormat { .object_store_registry .get_by_uri(&fmeta.path)? .file_reader(fmeta)? - .reader()?; + .sync_reader()?; let (schema, records_read) = arrow::csv::reader::infer_reader_schema( &mut reader, self.delimiter, diff --git a/datafusion/src/datasource/file_format/json.rs b/datafusion/src/datasource/file_format/json.rs index 20441b2d62c26..ffa6216cf7397 100644 --- a/datafusion/src/datasource/file_format/json.rs +++ b/datafusion/src/datasource/file_format/json.rs @@ -81,7 +81,7 @@ impl FileFormat for JsonFormat { .object_store_registry .get_by_uri(&fmeta.path)? .file_reader(fmeta)? - .reader()?; + .sync_reader()?; let mut reader = BufReader::new(reader); let iter = ValueIter::new(&mut reader, None); let schema = infer_json_schema_from_iterator(iter.take_while(|_| { diff --git a/datafusion/src/datasource/file_format/parquet.rs b/datafusion/src/datasource/file_format/parquet.rs index 99a8a890914f2..6e69651fc407f 100644 --- a/datafusion/src/datasource/file_format/parquet.rs +++ b/datafusion/src/datasource/file_format/parquet.rs @@ -337,7 +337,7 @@ impl ChunkReader for ChunkObjectReader { fn get_read(&self, start: u64, length: usize) -> ParquetResult { self.0 - .chunk_reader(start, length) + .sync_chunk_reader(start, length) .map_err(|e| ParquetError::ArrowError(e.to_string())) } } diff --git a/datafusion/src/datasource/object_store/local.rs b/datafusion/src/datasource/object_store/local.rs index 8fe46fad37aed..655e206274155 100644 --- a/datafusion/src/datasource/object_store/local.rs +++ b/datafusion/src/datasource/object_store/local.rs @@ -22,7 +22,7 @@ use std::io::{Read, Seek, SeekFrom}; use std::sync::Arc; use async_trait::async_trait; -use futures::{stream, StreamExt}; +use futures::{stream, AsyncRead, StreamExt}; use crate::datasource::object_store::{ ListEntryStream, ObjectReader, ObjectStore, SizedFile, SizedFileStream, @@ -65,7 +65,17 @@ impl LocalFileReader { #[async_trait] impl ObjectReader for LocalFileReader { - fn chunk_reader( + async fn chunk_reader( + &self, + _start: u64, + _length: usize, + ) -> Result> { + todo!( + "implement once async file readers are available (arrow-rs#78, arrow-rs#111)" + ) + } + + fn sync_chunk_reader( &self, start: u64, length: usize, diff --git a/datafusion/src/datasource/object_store/mod.rs b/datafusion/src/datasource/object_store/mod.rs index b03c795024b22..3474f691f4814 100644 --- a/datafusion/src/datasource/object_store/mod.rs +++ b/datafusion/src/datasource/object_store/mod.rs @@ -26,7 +26,7 @@ use std::pin::Pin; use std::sync::{Arc, RwLock}; use async_trait::async_trait; -use futures::{Stream, StreamExt}; +use futures::{AsyncRead, Stream, StreamExt}; use local::LocalFileSystem; @@ -37,16 +37,20 @@ use crate::error::{DataFusionError, Result}; /// have some performance impacts. #[async_trait] pub trait ObjectReader { + /// Get reader for a part [start, start + length] in the file asynchronously + async fn chunk_reader(&self, start: u64, length: usize) + -> Result>; + /// Get reader for a part [start, start + length] in the file - fn chunk_reader( + fn sync_chunk_reader( &self, start: u64, length: usize, ) -> Result>; /// Get reader for the entire file - fn reader(&self) -> Result> { - self.chunk_reader(0, self.length() as usize) + fn sync_reader(&self) -> Result> { + self.sync_chunk_reader(0, self.length() as usize) } /// Get the size of the file diff --git a/datafusion/src/physical_plan/file_format/avro.rs b/datafusion/src/physical_plan/file_format/avro.rs index e20ba3afe7a86..65b5b6e4b4e9f 100644 --- a/datafusion/src/physical_plan/file_format/avro.rs +++ b/datafusion/src/physical_plan/file_format/avro.rs @@ -129,7 +129,7 @@ impl ExecutionPlan for AvroExec { .object_store_registry .get_by_uri(&self.files[partition].file.path)? .file_reader(self.files[partition].file.clone())? - .reader()?; + .sync_reader()?; let proj = self.projection.as_ref().map(|p| { p.iter() diff --git a/datafusion/src/physical_plan/file_format/csv.rs b/datafusion/src/physical_plan/file_format/csv.rs index d7392579774ca..5aa7e62a112a0 100644 --- a/datafusion/src/physical_plan/file_format/csv.rs +++ b/datafusion/src/physical_plan/file_format/csv.rs @@ -196,7 +196,7 @@ impl CsvStream> { let file = object_store_registry .get_by_uri(&file.path)? .file_reader(file.clone())? - .reader()?; + .sync_reader()?; Self::try_new_from_reader( file, schema, has_header, delimiter, projection, batch_size, limit, ) diff --git a/datafusion/src/physical_plan/file_format/json.rs b/datafusion/src/physical_plan/file_format/json.rs index 1d02f5fca3b24..7a81a75fadc5e 100644 --- a/datafusion/src/physical_plan/file_format/json.rs +++ b/datafusion/src/physical_plan/file_format/json.rs @@ -129,7 +129,7 @@ impl ExecutionPlan for NdJsonExec { .object_store_registry .get_by_uri(&self.files[partition].file.path)? .file_reader(self.files[partition].file.clone())? - .reader()?; + .sync_reader()?; let json_reader = json::Reader::new(file, self.schema(), self.batch_size, proj); From bc829968c4c32133f2d78cced319f71e55ed78fc Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Mon, 4 Oct 2021 17:59:07 +0200 Subject: [PATCH 09/17] [doc] cleanup comments --- datafusion/src/datasource/file_format/mod.rs | 13 ++----------- 1 file changed, 2 insertions(+), 11 deletions(-) diff --git a/datafusion/src/datasource/file_format/mod.rs b/datafusion/src/datasource/file_format/mod.rs index b6e5f8a3f2777..a23816a7f0112 100644 --- a/datafusion/src/datasource/file_format/mod.rs +++ b/datafusion/src/datasource/file_format/mod.rs @@ -36,24 +36,15 @@ use futures::{Stream, StreamExt}; use super::object_store::{ObjectStoreRegistry, SizedFile, SizedFileStream}; -// /// A stream of String that can be used accross await calls -// pub type StringStream = Pin + Send + Sync>>; - -// /// Convert a vector into a stream -// pub fn string_stream(strings: Vec) -> StringStream { -// Box::pin(futures::stream::iter(strings)) -// } - /// This trait abstracts all the file format specific implementations /// from the `TableProvider`. This helps code re-utilization accross /// providers that support the the same file formats. #[async_trait] pub trait FileFormat: Send + Sync { - /// Open the files at the paths provided by iterator and infer the - /// common schema + /// Infer the the common schema of the files described by the path stream async fn infer_schema(&self, paths: SizedFileStream) -> Result; - /// Open the file at the given path and infer its statistics + /// Infer the statistics for the file at the given path async fn infer_stats(&self, path: SizedFile) -> Result; /// Take a list of files and convert it to the appropriate executor From 4bc9a42060d362e363329a64ba299534a790592d Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Mon, 4 Oct 2021 19:36:37 +0200 Subject: [PATCH 10/17] [test] test file listing --- datafusion/src/datasource/listing.rs | 139 ++++++++++++++++++++++----- 1 file changed, 115 insertions(+), 24 deletions(-) diff --git a/datafusion/src/datasource/listing.rs b/datafusion/src/datasource/listing.rs index bd5ba6804dc7c..a577299e9de6a 100644 --- a/datafusion/src/datasource/listing.rs +++ b/datafusion/src/datasource/listing.rs @@ -26,7 +26,7 @@ use futures::StreamExt; use crate::{ datasource::file_format::{self, PartitionedFile}, - error::Result, + error::{DataFusionError, Result}, logical_plan::Expr, physical_plan::{ExecutionPlan, Statistics}, }; @@ -126,6 +126,37 @@ impl TableProvider for ListingTable { filters: &[Expr], limit: Option, ) -> Result> { + let (partitioned_file_lists, statistics) = + self.list_files_for_scan(filters, limit).await?; + // create the execution plan + self.options + .format + .create_physical_plan( + self.schema(), + partitioned_file_lists, + statistics, + projection, + batch_size, + filters, + limit, + ) + .await + } + + fn supports_filter_pushdown( + &self, + _filter: &Expr, + ) -> Result { + Ok(TableProviderFilterPushDown::Inexact) + } +} + +impl ListingTable { + async fn list_files_for_scan( + &self, + filters: &[Expr], + limit: Option, + ) -> Result<(Vec>, Statistics)> { // list files (with partitions) let file_list = pruned_partition_list( self.options.format.object_store_registry(), @@ -153,28 +184,14 @@ impl TableProvider for ListingTable { let (files, statistics) = file_format::get_statistics_with_limit(files, self.schema(), limit).await?; - let partitioned_file_lists = split_files(files, self.options.max_partitions); - - // create the execution plan - self.options - .format - .create_physical_plan( - self.schema(), - partitioned_file_lists, - statistics, - projection, - batch_size, - filters, - limit, - ) - .await - } + if files.is_empty() { + return Err(DataFusionError::Plan(format!( + "No files found at {} with file extension {}", + self.path, self.options.file_extension, + ))); + } - fn supports_filter_pushdown( - &self, - _filter: &Expr, - ) -> Result { - Ok(TableProviderFilterPushDown::Inexact) + Ok((split_files(files, self.options.max_partitions), statistics)) } } @@ -217,7 +234,8 @@ fn split_files( #[cfg(test)] mod tests { use crate::datasource::{ - file_format::parquet::ParquetFormat, object_store::SizedFile, + file_format::{avro::AvroFormat, parquet::ParquetFormat}, + object_store::{ListEntryStream, ObjectStore, SizedFile, SizedFileStream}, }; use super::*; @@ -302,7 +320,14 @@ mod tests { Ok(()) } - // TODO add tests on listing once the ObjectStore abstraction is added + #[tokio::test] + async fn file_listings() -> Result<()> { + assert_partitioning(5, 12, 5).await?; + assert_partitioning(4, 4, 4).await?; + assert_partitioning(5, 2, 2).await?; + assert_partitioning(0, 2, 0).await.expect_err("no files"); + Ok(()) + } async fn load_table(name: &str) -> Result> { let testdata = crate::test_util::parquet_test_data(); @@ -319,4 +344,70 @@ mod tests { let table = ListingTable::new(&filename, schema, opt); Ok(Arc::new(table)) } + + async fn assert_partitioning( + files_in_folder: usize, + max_partitions: usize, + output_partitioning: usize, + ) -> Result<()> { + let registry = ObjectStoreRegistry::new(); + registry.register_store( + "mock".to_owned(), + Arc::new(MockObjectStore { files_in_folder }), + ); + + let format = AvroFormat::new(Arc::new(registry)); + + let opt = ListingOptions { + file_extension: "".to_owned(), + format: Arc::new(format), + partitions: vec![], + max_partitions, + collect_stat: true, + }; + + let schema = Schema::new(vec![Field::new("a", DataType::Boolean, false)]); + + let table = ListingTable::new("mock://bucket/key-prefix", Arc::new(schema), opt); + + let (file_list, _) = table.list_files_for_scan(&[], None).await?; + + assert_eq!(file_list.len(), output_partitioning); + + Ok(()) + } + + #[derive(Debug)] + struct MockObjectStore { + pub files_in_folder: usize, + } + + #[async_trait] + impl ObjectStore for MockObjectStore { + async fn list_file(&self, prefix: &str) -> Result { + let prefix = prefix.to_owned(); + let files = (0..self.files_in_folder).map(move |i| { + Ok(SizedFile { + path: format!("{}file{}", prefix, i), + size: 100, + }) + }); + Ok(Box::pin(futures::stream::iter(files))) + } + + async fn list_dir( + &self, + _prefix: &str, + _delimiter: Option, + ) -> Result { + unimplemented!() + } + + fn file_reader( + &self, + _file: SizedFile, + ) -> Result> { + unimplemented!() + } + } } From bb16ac168f98912dd4a461b89ad80737e8843e5e Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 5 Oct 2021 10:14:01 +0200 Subject: [PATCH 11/17] [fix] add last_modified back --- datafusion/src/datasource/listing.rs | 43 ++++++------------- .../src/datasource/object_store/local.rs | 5 ++- datafusion/src/datasource/object_store/mod.rs | 6 +++ 3 files changed, 23 insertions(+), 31 deletions(-) diff --git a/datafusion/src/datasource/listing.rs b/datafusion/src/datasource/listing.rs index a577299e9de6a..303c42503aa03 100644 --- a/datafusion/src/datasource/listing.rs +++ b/datafusion/src/datasource/listing.rs @@ -242,37 +242,19 @@ mod tests { #[test] fn test_split_files() { - let files = vec![ - PartitionedFile { - file: SizedFile { - path: "a".to_owned(), - size: 10, - }, - }, - PartitionedFile { - file: SizedFile { - path: "b".to_owned(), - size: 10, - }, - }, - PartitionedFile { - file: SizedFile { - path: "c".to_owned(), - size: 10, - }, - }, - PartitionedFile { - file: SizedFile { - path: "d".to_owned(), - size: 10, - }, - }, - PartitionedFile { - file: SizedFile { - path: "e".to_owned(), - size: 10, - }, + let new_partitioned_file = |path: &str| PartitionedFile { + file: SizedFile { + path: path.to_owned(), + size: 10, + last_modified: None, }, + }; + let files = vec![ + new_partitioned_file("a"), + new_partitioned_file("b"), + new_partitioned_file("c"), + new_partitioned_file("d"), + new_partitioned_file("e"), ]; let chunks = split_files(files.clone(), 1); @@ -390,6 +372,7 @@ mod tests { Ok(SizedFile { path: format!("{}file{}", prefix, i), size: 100, + last_modified: None, }) }); Ok(Box::pin(futures::stream::iter(files))) diff --git a/datafusion/src/datasource/object_store/local.rs b/datafusion/src/datasource/object_store/local.rs index 655e206274155..d9006e2b5a767 100644 --- a/datafusion/src/datasource/object_store/local.rs +++ b/datafusion/src/datasource/object_store/local.rs @@ -97,6 +97,7 @@ async fn list_all(prefix: String) -> Result { SizedFile { path, size: metadata.len(), + last_modified: metadata.modified().map(chrono::DateTime::from).ok(), } } @@ -154,9 +155,11 @@ pub fn local_sized_file_stream(files: Vec) -> SizedFileStream { /// Helper method to fetch the file size at given path and create a `SizedFile` pub fn local_sized_file(file: String) -> SizedFile { + let metadata = fs::metadata(&file).expect("Local file metadata"); SizedFile { - size: fs::metadata(&file).expect("Local file metadata").len(), + size: metadata.len(), path: file, + last_modified: metadata.modified().map(chrono::DateTime::from).ok(), } } diff --git a/datafusion/src/datasource/object_store/mod.rs b/datafusion/src/datasource/object_store/mod.rs index 3474f691f4814..20e43021a8fc5 100644 --- a/datafusion/src/datasource/object_store/mod.rs +++ b/datafusion/src/datasource/object_store/mod.rs @@ -26,6 +26,7 @@ use std::pin::Pin; use std::sync::{Arc, RwLock}; use async_trait::async_trait; +use chrono::{DateTime, Utc}; use futures::{AsyncRead, Stream, StreamExt}; use local::LocalFileSystem; @@ -73,6 +74,11 @@ pub struct SizedFile { pub path: String, /// File size in total pub size: u64, + /// The last modification time of the file according to the + /// object store metadata. This information might be used by + /// catalog systems like Delta Lake for time travel (see + /// https://github.com/delta-io/delta/issues/192) + pub last_modified: Option>, } impl std::fmt::Display for SizedFile { From cd11131996b68e6c930f1da6b80c504855a2399c Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 5 Oct 2021 10:20:34 +0200 Subject: [PATCH 12/17] [refacto] simplify csv reader exec --- .../src/physical_plan/file_format/csv.rs | 35 +++++-------------- 1 file changed, 9 insertions(+), 26 deletions(-) diff --git a/datafusion/src/physical_plan/file_format/csv.rs b/datafusion/src/physical_plan/file_format/csv.rs index 5aa7e62a112a0..9df7a3bef3c7f 100644 --- a/datafusion/src/physical_plan/file_format/csv.rs +++ b/datafusion/src/physical_plan/file_format/csv.rs @@ -18,7 +18,7 @@ //! Execution plan for reading CSV files use crate::datasource::file_format::PartitionedFile; -use crate::datasource::object_store::{ObjectStoreRegistry, SizedFile}; +use crate::datasource::object_store::ObjectStoreRegistry; use crate::error::{DataFusionError, Result}; use crate::physical_plan::{ DisplayFormatType, ExecutionPlan, Partitioning, RecordBatchStream, @@ -135,9 +135,14 @@ impl ExecutionPlan for CsvExec { } async fn execute(&self, partition: usize) -> Result { - Ok(Box::pin(CsvStream::try_new( - &self.object_store_registry, - &self.files[partition].file, + let file = self + .object_store_registry + .get_by_uri(&self.files[partition].file.path)? + .file_reader(self.files[partition].file.clone())? + .sync_reader()?; + + Ok(Box::pin(CsvStream::try_new_from_reader( + file, self.schema.clone(), self.has_header, self.delimiter, @@ -180,28 +185,6 @@ struct CsvStream { /// Arrow CSV reader reader: csv::Reader, } -impl CsvStream> { - /// Create an iterator for a CSV file - #[allow(clippy::too_many_arguments)] - pub fn try_new( - object_store_registry: &ObjectStoreRegistry, - file: &SizedFile, - schema: SchemaRef, - has_header: bool, - delimiter: Option, - projection: &Option>, - batch_size: usize, - limit: Option, - ) -> Result { - let file = object_store_registry - .get_by_uri(&file.path)? - .file_reader(file.clone())? - .sync_reader()?; - Self::try_new_from_reader( - file, schema, has_header, delimiter, projection, batch_size, limit, - ) - } -} impl CsvStream { /// Create an iterator for a reader From df4eb663960019f1fc7b741e749531f6fef5101f Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 5 Oct 2021 10:28:38 +0200 Subject: [PATCH 13/17] [refacto] change SizedFile back to FileMeta --- datafusion/src/datasource/file_format/avro.rs | 18 +++++------ datafusion/src/datasource/file_format/csv.rs | 14 ++++----- datafusion/src/datasource/file_format/json.rs | 16 +++++----- datafusion/src/datasource/file_format/mod.rs | 8 ++--- .../src/datasource/file_format/parquet.rs | 18 +++++------ datafusion/src/datasource/listing.rs | 10 +++---- .../src/datasource/object_store/local.rs | 30 +++++++++---------- datafusion/src/datasource/object_store/mod.rs | 16 +++++----- .../src/physical_plan/file_format/avro.rs | 6 ++-- .../src/physical_plan/file_format/csv.rs | 6 ++-- .../src/physical_plan/file_format/json.rs | 8 ++--- .../src/physical_plan/file_format/parquet.rs | 6 ++-- 12 files changed, 78 insertions(+), 78 deletions(-) diff --git a/datafusion/src/datasource/file_format/avro.rs b/datafusion/src/datasource/file_format/avro.rs index ac83d63296ae1..8f46affe8656b 100644 --- a/datafusion/src/datasource/file_format/avro.rs +++ b/datafusion/src/datasource/file_format/avro.rs @@ -27,7 +27,7 @@ use futures::StreamExt; use super::FileFormat; use super::PartitionedFile; use crate::avro_to_arrow::read_avro_schema_from_reader; -use crate::datasource::object_store::{ObjectStoreRegistry, SizedFile, SizedFileStream}; +use crate::datasource::object_store::{FileMeta, FileMetaStream, ObjectStoreRegistry}; use crate::error::Result; use crate::logical_plan::Expr; use crate::physical_plan::file_format::AvroExec; @@ -59,7 +59,7 @@ impl AvroFormat { #[async_trait] impl FileFormat for AvroFormat { - async fn infer_schema(&self, mut file_stream: SizedFileStream) -> Result { + async fn infer_schema(&self, mut file_stream: FileMetaStream) -> Result { let mut schemas = vec![]; while let Some(fmeta_res) = file_stream.next().await { let fmeta = fmeta_res?; @@ -75,7 +75,7 @@ impl FileFormat for AvroFormat { Ok(Arc::new(merged_schema)) } - async fn infer_stats(&self, _path: SizedFile) -> Result { + async fn infer_stats(&self, _path: FileMeta) -> Result { Ok(Statistics::default()) } @@ -111,7 +111,7 @@ impl FileFormat for AvroFormat { #[cfg(feature = "avro")] mod tests { use crate::{ - datasource::object_store::local::{local_sized_file, local_sized_file_stream}, + datasource::object_store::local::{local_file_meta, local_file_meta_stream}, physical_plan::collect, }; @@ -376,15 +376,15 @@ mod tests { let filename = format!("{}/avro/{}", testdata, file_name); let format = AvroFormat::default(); let schema = format - .infer_schema(local_sized_file_stream(vec![filename.clone()])) + .infer_schema(local_file_meta_stream(vec![filename.clone()])) .await .expect("Schema inference"); let stats = format - .infer_stats(local_sized_file(filename.clone())) + .infer_stats(local_file_meta(filename.clone())) .await .expect("Stats inference"); let files = vec![vec![PartitionedFile { - file: local_sized_file(filename.to_owned()), + file: local_file_meta(filename.to_owned()), }]]; let exec = format .create_physical_plan( @@ -406,7 +406,7 @@ mod tests { mod tests { use super::*; - use crate::datasource::object_store::local::local_sized_file_stream; + use crate::datasource::object_store::local::local_file_meta_stream; use crate::error::DataFusionError; #[tokio::test] @@ -414,7 +414,7 @@ mod tests { let testdata = crate::test_util::arrow_test_data(); let filename = format!("{}/avro/alltypes_plain.avro", testdata); let schema_result = AvroFormat::default() - .infer_schema(local_sized_file_stream(vec![filename])) + .infer_schema(local_file_meta_stream(vec![filename])) .await; assert!(matches!( schema_result, diff --git a/datafusion/src/datasource/file_format/csv.rs b/datafusion/src/datasource/file_format/csv.rs index 302afb4c0d9c3..ea47223cc60a9 100644 --- a/datafusion/src/datasource/file_format/csv.rs +++ b/datafusion/src/datasource/file_format/csv.rs @@ -26,7 +26,7 @@ use futures::StreamExt; use super::FileFormat; use super::PartitionedFile; -use crate::datasource::object_store::{ObjectStoreRegistry, SizedFile, SizedFileStream}; +use crate::datasource::object_store::{FileMeta, FileMetaStream, ObjectStoreRegistry}; use crate::error::Result; use crate::logical_plan::Expr; use crate::physical_plan::file_format::CsvExec; @@ -85,7 +85,7 @@ impl CsvFormat { #[async_trait] impl FileFormat for CsvFormat { - async fn infer_schema(&self, mut file_stream: SizedFileStream) -> Result { + async fn infer_schema(&self, mut file_stream: FileMetaStream) -> Result { let mut schemas = vec![]; let mut records_to_read = self.schema_infer_max_rec.unwrap_or(std::usize::MAX); @@ -116,7 +116,7 @@ impl FileFormat for CsvFormat { Ok(Arc::new(merged_schema)) } - async fn infer_stats(&self, _path: SizedFile) -> Result { + async fn infer_stats(&self, _path: FileMeta) -> Result { Ok(Statistics::default()) } @@ -156,7 +156,7 @@ mod tests { use super::*; use crate::{ - datasource::object_store::local::{local_sized_file, local_sized_file_stream}, + datasource::object_store::local::{local_file_meta, local_file_meta_stream}, physical_plan::collect, }; @@ -266,15 +266,15 @@ mod tests { let filename = format!("{}/csv/{}", testdata, file_name); let format = CsvFormat::default(); let schema = format - .infer_schema(local_sized_file_stream(vec![filename.clone()])) + .infer_schema(local_file_meta_stream(vec![filename.clone()])) .await .expect("Schema inference"); let stats = format - .infer_stats(local_sized_file(filename.clone())) + .infer_stats(local_file_meta(filename.clone())) .await .expect("Stats inference"); let files = vec![vec![PartitionedFile { - file: local_sized_file(filename.to_owned()), + file: local_file_meta(filename.to_owned()), }]]; let exec = format .create_physical_plan( diff --git a/datafusion/src/datasource/file_format/json.rs b/datafusion/src/datasource/file_format/json.rs index ffa6216cf7397..f7ed8517ae636 100644 --- a/datafusion/src/datasource/file_format/json.rs +++ b/datafusion/src/datasource/file_format/json.rs @@ -29,9 +29,9 @@ use futures::StreamExt; use super::FileFormat; use super::PartitionedFile; +use crate::datasource::object_store::FileMeta; +use crate::datasource::object_store::FileMetaStream; use crate::datasource::object_store::ObjectStoreRegistry; -use crate::datasource::object_store::SizedFile; -use crate::datasource::object_store::SizedFileStream; use crate::error::Result; use crate::logical_plan::Expr; use crate::physical_plan::file_format::NdJsonExec; @@ -72,7 +72,7 @@ impl JsonFormat { #[async_trait] impl FileFormat for JsonFormat { - async fn infer_schema(&self, mut file_stream: SizedFileStream) -> Result { + async fn infer_schema(&self, mut file_stream: FileMetaStream) -> Result { let mut schemas = Vec::new(); let mut records_to_read = self.schema_infer_max_rec.unwrap_or(usize::MAX); while let Some(fmeta_res) = file_stream.next().await { @@ -99,7 +99,7 @@ impl FileFormat for JsonFormat { Ok(Arc::new(schema)) } - async fn infer_stats(&self, _path: SizedFile) -> Result { + async fn infer_stats(&self, _path: FileMeta) -> Result { Ok(Statistics::default()) } @@ -137,7 +137,7 @@ mod tests { use super::*; use crate::{ - datasource::object_store::local::{local_sized_file, local_sized_file_stream}, + datasource::object_store::local::{local_file_meta, local_file_meta_stream}, physical_plan::collect, }; @@ -230,15 +230,15 @@ mod tests { let filename = "tests/jsons/2.json"; let format = JsonFormat::default(); let schema = format - .infer_schema(local_sized_file_stream(vec![filename.to_owned()])) + .infer_schema(local_file_meta_stream(vec![filename.to_owned()])) .await .expect("Schema inference"); let stats = format - .infer_stats(local_sized_file(filename.to_owned())) + .infer_stats(local_file_meta(filename.to_owned())) .await .expect("Stats inference"); let files = vec![vec![PartitionedFile { - file: local_sized_file(filename.to_owned()), + file: local_file_meta(filename.to_owned()), }]]; let exec = format .create_physical_plan( diff --git a/datafusion/src/datasource/file_format/mod.rs b/datafusion/src/datasource/file_format/mod.rs index a23816a7f0112..a876cac5e4e7f 100644 --- a/datafusion/src/datasource/file_format/mod.rs +++ b/datafusion/src/datasource/file_format/mod.rs @@ -34,7 +34,7 @@ use crate::physical_plan::{Accumulator, ExecutionPlan, Statistics}; use async_trait::async_trait; use futures::{Stream, StreamExt}; -use super::object_store::{ObjectStoreRegistry, SizedFile, SizedFileStream}; +use super::object_store::{FileMeta, FileMetaStream, ObjectStoreRegistry}; /// This trait abstracts all the file format specific implementations /// from the `TableProvider`. This helps code re-utilization accross @@ -42,10 +42,10 @@ use super::object_store::{ObjectStoreRegistry, SizedFile, SizedFileStream}; #[async_trait] pub trait FileFormat: Send + Sync { /// Infer the the common schema of the files described by the path stream - async fn infer_schema(&self, paths: SizedFileStream) -> Result; + async fn infer_schema(&self, paths: FileMetaStream) -> Result; /// Infer the statistics for the file at the given path - async fn infer_stats(&self, path: SizedFile) -> Result; + async fn infer_stats(&self, path: FileMeta) -> Result; /// Take a list of files and convert it to the appropriate executor /// according to this file format. @@ -159,7 +159,7 @@ pub async fn get_statistics_with_limit( /// TODO move back to crate::datasource::mod.rs once legacy cleaned up pub struct PartitionedFile { /// Path for the file (e.g. URL, filesystem path, etc) - pub file: SizedFile, + pub file: FileMeta, // Values of partition columns to be appended to each row // pub partition_value: Option>, // We may include row group range here for a more fine-grained parallel execution diff --git a/datafusion/src/datasource/file_format/parquet.rs b/datafusion/src/datasource/file_format/parquet.rs index 6e69651fc407f..da132c5d76683 100644 --- a/datafusion/src/datasource/file_format/parquet.rs +++ b/datafusion/src/datasource/file_format/parquet.rs @@ -37,10 +37,10 @@ use super::FileFormat; use super::PartitionedFile; use super::{create_max_min_accs, get_col_stats}; use crate::arrow::datatypes::{DataType, Field}; +use crate::datasource::object_store::FileMeta; +use crate::datasource::object_store::FileMetaStream; use crate::datasource::object_store::ObjectReader; use crate::datasource::object_store::ObjectStoreRegistry; -use crate::datasource::object_store::SizedFile; -use crate::datasource::object_store::SizedFileStream; use crate::error::DataFusionError; use crate::error::Result; use crate::logical_plan::combine_filters; @@ -85,7 +85,7 @@ impl ParquetFormat { #[async_trait] impl FileFormat for ParquetFormat { - async fn infer_schema(&self, mut paths: SizedFileStream) -> Result { + async fn infer_schema(&self, mut paths: FileMetaStream) -> Result { // We currently get the schema information from the first file rather than do // schema merging and this is a limitation. // See https://issues.apache.org/jira/browse/ARROW-11017 @@ -97,7 +97,7 @@ impl FileFormat for ParquetFormat { Ok(Arc::new(schema)) } - async fn infer_stats(&self, path: SizedFile) -> Result { + async fn infer_stats(&self, path: FileMeta) -> Result { let (_, stats) = fetch_metadata(&self.object_store_registry, path)?; Ok(stats) } @@ -263,7 +263,7 @@ fn summarize_min_max( /// Read and parse the metadata of the Parquet file at location `path` fn fetch_metadata( object_store_registry: &ObjectStoreRegistry, - fmeta: SizedFile, + fmeta: FileMeta, ) -> Result<(Schema, Statistics)> { let object_store = object_store_registry.get_by_uri(&fmeta.path)?; let obj_reader = ChunkObjectReader(object_store.file_reader(fmeta)?); @@ -345,7 +345,7 @@ impl ChunkReader for ChunkObjectReader { #[cfg(test)] mod tests { use crate::{ - datasource::object_store::local::{local_sized_file, local_sized_file_stream}, + datasource::object_store::local::{local_file_meta, local_file_meta_stream}, physical_plan::collect, }; @@ -605,15 +605,15 @@ mod tests { let filename = format!("{}/{}", testdata, file_name); let format = ParquetFormat::default(); let schema = format - .infer_schema(local_sized_file_stream(vec![filename.clone()])) + .infer_schema(local_file_meta_stream(vec![filename.clone()])) .await .expect("Schema inference"); let stats = format - .infer_stats(local_sized_file(filename.clone())) + .infer_stats(local_file_meta(filename.clone())) .await .expect("Stats inference"); let files = vec![vec![PartitionedFile { - file: local_sized_file(filename.clone()), + file: local_file_meta(filename.clone()), }]]; let exec = format .create_physical_plan( diff --git a/datafusion/src/datasource/listing.rs b/datafusion/src/datasource/listing.rs index 303c42503aa03..ff28d3b214701 100644 --- a/datafusion/src/datasource/listing.rs +++ b/datafusion/src/datasource/listing.rs @@ -235,7 +235,7 @@ fn split_files( mod tests { use crate::datasource::{ file_format::{avro::AvroFormat, parquet::ParquetFormat}, - object_store::{ListEntryStream, ObjectStore, SizedFile, SizedFileStream}, + object_store::{FileMeta, FileMetaStream, ListEntryStream, ObjectStore}, }; use super::*; @@ -243,7 +243,7 @@ mod tests { #[test] fn test_split_files() { let new_partitioned_file = |path: &str| PartitionedFile { - file: SizedFile { + file: FileMeta { path: path.to_owned(), size: 10, last_modified: None, @@ -366,10 +366,10 @@ mod tests { #[async_trait] impl ObjectStore for MockObjectStore { - async fn list_file(&self, prefix: &str) -> Result { + async fn list_file(&self, prefix: &str) -> Result { let prefix = prefix.to_owned(); let files = (0..self.files_in_folder).map(move |i| { - Ok(SizedFile { + Ok(FileMeta { path: format!("{}file{}", prefix, i), size: 100, last_modified: None, @@ -388,7 +388,7 @@ mod tests { fn file_reader( &self, - _file: SizedFile, + _file: FileMeta, ) -> Result> { unimplemented!() } diff --git a/datafusion/src/datasource/object_store/local.rs b/datafusion/src/datasource/object_store/local.rs index d9006e2b5a767..2a331d8e6f154 100644 --- a/datafusion/src/datasource/object_store/local.rs +++ b/datafusion/src/datasource/object_store/local.rs @@ -25,7 +25,7 @@ use async_trait::async_trait; use futures::{stream, AsyncRead, StreamExt}; use crate::datasource::object_store::{ - ListEntryStream, ObjectReader, ObjectStore, SizedFile, SizedFileStream, + FileMeta, FileMetaStream, ListEntryStream, ObjectReader, ObjectStore, }; use crate::error::DataFusionError; use crate::error::Result; @@ -36,7 +36,7 @@ pub struct LocalFileSystem; #[async_trait] impl ObjectStore for LocalFileSystem { - async fn list_file(&self, prefix: &str) -> Result { + async fn list_file(&self, prefix: &str) -> Result { list_all(prefix.to_owned()).await } @@ -48,17 +48,17 @@ impl ObjectStore for LocalFileSystem { todo!() } - fn file_reader(&self, file: SizedFile) -> Result> { + fn file_reader(&self, file: FileMeta) -> Result> { Ok(Arc::new(LocalFileReader::new(file)?)) } } struct LocalFileReader { - file: SizedFile, + file: FileMeta, } impl LocalFileReader { - fn new(file: SizedFile) -> Result { + fn new(file: FileMeta) -> Result { Ok(Self { file }) } } @@ -92,9 +92,9 @@ impl ObjectReader for LocalFileReader { } } -async fn list_all(prefix: String) -> Result { - fn get_meta(path: String, metadata: Metadata) -> SizedFile { - SizedFile { +async fn list_all(prefix: String) -> Result { + fn get_meta(path: String, metadata: Metadata) -> FileMeta { + FileMeta { path, size: metadata.len(), last_modified: metadata.modified().map(chrono::DateTime::from).ok(), @@ -104,7 +104,7 @@ async fn list_all(prefix: String) -> Result { async fn find_files_in_dir( path: String, to_visit: &mut Vec, - ) -> Result> { + ) -> Result> { let mut dir = tokio::fs::read_dir(path).await?; let mut files = Vec::new(); @@ -148,15 +148,15 @@ async fn list_all(prefix: String) -> Result { } } -/// Create a stream of `SizedFile` applying `local_sized_file` to each path in `files` -pub fn local_sized_file_stream(files: Vec) -> SizedFileStream { - Box::pin(futures::stream::iter(files).map(|f| Ok(local_sized_file(f)))) +/// Create a stream of `FileMeta` applying `local_file_meta` to each path in `files` +pub fn local_file_meta_stream(files: Vec) -> FileMetaStream { + Box::pin(futures::stream::iter(files).map(|f| Ok(local_file_meta(f)))) } -/// Helper method to fetch the file size at given path and create a `SizedFile` -pub fn local_sized_file(file: String) -> SizedFile { +/// Helper method to fetch the file size at given path and create a `FileMeta` +pub fn local_file_meta(file: String) -> FileMeta { let metadata = fs::metadata(&file).expect("Local file metadata"); - SizedFile { + FileMeta { size: metadata.len(), path: file, last_modified: metadata.modified().map(chrono::DateTime::from).ok(), diff --git a/datafusion/src/datasource/object_store/mod.rs b/datafusion/src/datasource/object_store/mod.rs index 20e43021a8fc5..7a6f024595ab3 100644 --- a/datafusion/src/datasource/object_store/mod.rs +++ b/datafusion/src/datasource/object_store/mod.rs @@ -62,14 +62,14 @@ pub trait ObjectReader { #[derive(Debug)] pub enum ListEntry { /// Complete file path with size - SizedFile(SizedFile), + FileMeta(FileMeta), /// Prefix to be further resolved during partition discovery Prefix(String), } /// Complete file path with size we got from object store #[derive(Debug, Clone)] -pub struct SizedFile { +pub struct FileMeta { /// Path of the file pub path: String, /// File size in total @@ -81,15 +81,15 @@ pub struct SizedFile { pub last_modified: Option>, } -impl std::fmt::Display for SizedFile { +impl std::fmt::Display for FileMeta { fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { write!(f, "{} (size: {})", self.path, self.size) } } /// Stream of files listed from object store -pub type SizedFileStream = - Pin> + Send + Sync + 'static>>; +pub type FileMetaStream = + Pin> + Send + Sync + 'static>>; /// Stream of list entries obtained from object store pub type ListEntryStream = @@ -100,14 +100,14 @@ pub type ListEntryStream = #[async_trait] pub trait ObjectStore: Sync + Send + Debug { /// Returns all the files in path `prefix` - async fn list_file(&self, prefix: &str) -> Result; + async fn list_file(&self, prefix: &str) -> Result; /// Calls `list_file` with a suffix filter async fn list_file_with_suffix( &self, prefix: &str, suffix: &str, - ) -> Result { + ) -> Result { let file_stream = self.list_file(prefix).await?; let suffix = suffix.to_owned(); Ok(Box::pin(file_stream.filter(move |fr| { @@ -128,7 +128,7 @@ pub trait ObjectStore: Sync + Send + Debug { ) -> Result; /// Get object reader for one file - fn file_reader(&self, file: SizedFile) -> Result>; + fn file_reader(&self, file: FileMeta) -> Result>; } static LOCAL_SCHEME: &str = "file"; diff --git a/datafusion/src/physical_plan/file_format/avro.rs b/datafusion/src/physical_plan/file_format/avro.rs index 65b5b6e4b4e9f..d079748ec85f7 100644 --- a/datafusion/src/physical_plan/file_format/avro.rs +++ b/datafusion/src/physical_plan/file_format/avro.rs @@ -244,7 +244,7 @@ impl RecordBatchStream for AvroStream<'_, R> { mod tests { use crate::datasource::object_store::local::{ - local_sized_file, local_sized_file_stream, + local_file_meta, local_file_meta_stream, }; use super::*; @@ -260,11 +260,11 @@ mod tests { let avro_exec = AvroExec::new( Arc::new(ObjectStoreRegistry::new()), vec![PartitionedFile { - file: local_sized_file(filename.clone()), + file: local_file_meta(filename.clone()), }], Statistics::default(), AvroFormat::default() - .infer_schema(local_sized_file_stream(vec![filename])) + .infer_schema(local_file_meta_stream(vec![filename])) .await?, Some(vec![0, 1, 2]), 1024, diff --git a/datafusion/src/physical_plan/file_format/csv.rs b/datafusion/src/physical_plan/file_format/csv.rs index 9df7a3bef3c7f..e662f7cecbec9 100644 --- a/datafusion/src/physical_plan/file_format/csv.rs +++ b/datafusion/src/physical_plan/file_format/csv.rs @@ -236,7 +236,7 @@ impl RecordBatchStream for CsvStream { mod tests { use super::*; use crate::{ - datasource::object_store::local::local_sized_file, test::aggr_test_schema, + datasource::object_store::local::local_file_meta, test::aggr_test_schema, }; use futures::StreamExt; @@ -249,7 +249,7 @@ mod tests { let csv = CsvExec::new( Arc::new(ObjectStoreRegistry::new()), vec![PartitionedFile { - file: local_sized_file(path), + file: local_file_meta(path), }], Statistics::default(), schema, @@ -282,7 +282,7 @@ mod tests { let csv = CsvExec::new( Arc::new(ObjectStoreRegistry::new()), vec![PartitionedFile { - file: local_sized_file(path), + file: local_file_meta(path), }], Statistics::default(), schema, diff --git a/datafusion/src/physical_plan/file_format/json.rs b/datafusion/src/physical_plan/file_format/json.rs index 7a81a75fadc5e..4e62c5e298405 100644 --- a/datafusion/src/physical_plan/file_format/json.rs +++ b/datafusion/src/physical_plan/file_format/json.rs @@ -229,7 +229,7 @@ mod tests { use crate::datasource::{ file_format::{json::JsonFormat, FileFormat}, - object_store::local::{local_sized_file, local_sized_file_stream}, + object_store::local::{local_file_meta, local_file_meta_stream}, }; use super::*; @@ -238,7 +238,7 @@ mod tests { async fn infer_schema(path: String) -> Result { JsonFormat::default() - .infer_schema(local_sized_file_stream(vec![path])) + .infer_schema(local_file_meta_stream(vec![path])) .await } @@ -249,7 +249,7 @@ mod tests { let exec = NdJsonExec::new( Arc::new(ObjectStoreRegistry::new()), vec![PartitionedFile { - file: local_sized_file(path.clone()), + file: local_file_meta(path.clone()), }], Default::default(), infer_schema(path).await?, @@ -304,7 +304,7 @@ mod tests { let exec = NdJsonExec::new( Arc::new(ObjectStoreRegistry::new()), vec![PartitionedFile { - file: local_sized_file(path.clone()), + file: local_file_meta(path.clone()), }], Default::default(), infer_schema(path).await?, diff --git a/datafusion/src/physical_plan/file_format/parquet.rs b/datafusion/src/physical_plan/file_format/parquet.rs index 433182abb7bf6..1be3d8b3284a3 100644 --- a/datafusion/src/physical_plan/file_format/parquet.rs +++ b/datafusion/src/physical_plan/file_format/parquet.rs @@ -543,7 +543,7 @@ fn read_partition( mod tests { use crate::datasource::{ file_format::{parquet::ParquetFormat, FileFormat}, - object_store::local::{local_sized_file, local_sized_file_stream}, + object_store::local::{local_file_meta, local_file_meta_stream}, }; use super::*; @@ -562,11 +562,11 @@ mod tests { let parquet_exec = ParquetExec::new( Arc::new(ObjectStoreRegistry::new()), vec![vec![PartitionedFile { - file: local_sized_file(filename.clone()), + file: local_file_meta(filename.clone()), }]], Statistics::default(), ParquetFormat::default() - .infer_schema(local_sized_file_stream(vec![filename])) + .infer_schema(local_file_meta_stream(vec![filename])) .await?, Some(vec![0, 1, 2]), None, From 64a510ceb39f0446107c21598a00dfde7267a748 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 5 Oct 2021 10:48:35 +0200 Subject: [PATCH 14/17] [doc] comment clarification --- datafusion/src/datasource/object_store/mod.rs | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/datafusion/src/datasource/object_store/mod.rs b/datafusion/src/datasource/object_store/mod.rs index 7a6f024595ab3..6b932f1c4ee3d 100644 --- a/datafusion/src/datasource/object_store/mod.rs +++ b/datafusion/src/datasource/object_store/mod.rs @@ -58,10 +58,11 @@ pub trait ObjectReader { fn length(&self) -> u64; } -/// Represents a file or a prefix that may require further resolution +/// Represents a specific file or a prefix (folder) that may +/// require further resolution #[derive(Debug)] pub enum ListEntry { - /// Complete file path with size + /// Specific file with metadata FileMeta(FileMeta), /// Prefix to be further resolved during partition discovery Prefix(String), From ce6022d984d5601cb1ef0270a99a6ea29d3c1e53 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 5 Oct 2021 16:07:27 +0200 Subject: [PATCH 15/17] [fix] avoid keeping object store as field --- datafusion/src/datasource/file_format/avro.rs | 64 +++---- datafusion/src/datasource/file_format/csv.rs | 45 ++--- datafusion/src/datasource/file_format/json.rs | 47 ++--- datafusion/src/datasource/file_format/mod.rs | 18 +- .../src/datasource/file_format/parquet.rs | 52 ++---- datafusion/src/datasource/listing.rs | 164 ++++++++++++------ .../src/datasource/object_store/local.rs | 39 +++-- datafusion/src/datasource/object_store/mod.rs | 44 ++++- .../src/physical_plan/file_format/avro.rs | 25 ++- .../src/physical_plan/file_format/csv.rs | 26 +-- .../src/physical_plan/file_format/json.rs | 29 ++-- .../src/physical_plan/file_format/parquet.rs | 31 ++-- 12 files changed, 307 insertions(+), 277 deletions(-) diff --git a/datafusion/src/datasource/file_format/avro.rs b/datafusion/src/datasource/file_format/avro.rs index 8f46affe8656b..cfe15090e23ff 100644 --- a/datafusion/src/datasource/file_format/avro.rs +++ b/datafusion/src/datasource/file_format/avro.rs @@ -27,7 +27,7 @@ use futures::StreamExt; use super::FileFormat; use super::PartitionedFile; use crate::avro_to_arrow::read_avro_schema_from_reader; -use crate::datasource::object_store::{FileMeta, FileMetaStream, ObjectStoreRegistry}; +use crate::datasource::object_store::{ObjectReader, ObjectReaderStream, ObjectStore}; use crate::error::Result; use crate::logical_plan::Expr; use crate::physical_plan::file_format::AvroExec; @@ -35,39 +35,14 @@ use crate::physical_plan::ExecutionPlan; use crate::physical_plan::Statistics; /// Avro `FileFormat` implementation. -pub struct AvroFormat { - /// Object store registry - pub object_store_registry: Arc, -} - -impl Default for AvroFormat { - fn default() -> Self { - Self { - object_store_registry: Arc::new(ObjectStoreRegistry::new()), - } - } -} - -impl AvroFormat { - /// Create Parquet with the given object store and default values - pub fn new(object_store_registry: Arc) -> Self { - Self { - object_store_registry, - } - } -} +pub struct AvroFormat; #[async_trait] impl FileFormat for AvroFormat { - async fn infer_schema(&self, mut file_stream: FileMetaStream) -> Result { + async fn infer_schema(&self, mut readers: ObjectReaderStream) -> Result { let mut schemas = vec![]; - while let Some(fmeta_res) = file_stream.next().await { - let fmeta = fmeta_res?; - let mut reader = self - .object_store_registry - .get_by_uri(&fmeta.path)? - .file_reader(fmeta)? - .sync_reader()?; + while let Some(obj_reader) = readers.next().await { + let mut reader = obj_reader?.sync_reader()?; let schema = read_avro_schema_from_reader(&mut reader)?; schemas.push(schema); } @@ -75,12 +50,13 @@ impl FileFormat for AvroFormat { Ok(Arc::new(merged_schema)) } - async fn infer_stats(&self, _path: FileMeta) -> Result { + async fn infer_stats(&self, _reader: Arc) -> Result { Ok(Statistics::default()) } async fn create_physical_plan( &self, + object_store: Arc, schema: SchemaRef, files: Vec>, statistics: Statistics, @@ -90,7 +66,7 @@ impl FileFormat for AvroFormat { limit: Option, ) -> Result> { let exec = AvroExec::new( - Arc::clone(&self.object_store_registry), + object_store, // flattening this for now because CsvExec does not support partitioning yet files.into_iter().flatten().collect(), statistics, @@ -101,17 +77,16 @@ impl FileFormat for AvroFormat { ); Ok(Arc::new(exec)) } - - fn object_store_registry(&self) -> &Arc { - &self.object_store_registry - } } #[cfg(test)] #[cfg(feature = "avro")] mod tests { use crate::{ - datasource::object_store::local::{local_file_meta, local_file_meta_stream}, + datasource::object_store::local::{ + local_file_meta, local_object_reader, local_object_reader_stream, + LocalFileSystem, + }, physical_plan::collect, }; @@ -374,20 +349,21 @@ mod tests { ) -> Result> { let testdata = crate::test_util::arrow_test_data(); let filename = format!("{}/avro/{}", testdata, file_name); - let format = AvroFormat::default(); + let format = AvroFormat {}; let schema = format - .infer_schema(local_file_meta_stream(vec![filename.clone()])) + .infer_schema(local_object_reader_stream(vec![filename.clone()])) .await .expect("Schema inference"); let stats = format - .infer_stats(local_file_meta(filename.clone())) + .infer_stats(local_object_reader(filename.clone())) .await .expect("Stats inference"); let files = vec![vec![PartitionedFile { - file: local_file_meta(filename.to_owned()), + file_meta: local_file_meta(filename.to_owned()), }]]; let exec = format .create_physical_plan( + Arc::new(LocalFileSystem {}), schema, files, stats, @@ -406,15 +382,15 @@ mod tests { mod tests { use super::*; - use crate::datasource::object_store::local::local_file_meta_stream; + use crate::datasource::object_store::local::local_object_reader_stream; use crate::error::DataFusionError; #[tokio::test] async fn test() -> Result<()> { let testdata = crate::test_util::arrow_test_data(); let filename = format!("{}/avro/alltypes_plain.avro", testdata); - let schema_result = AvroFormat::default() - .infer_schema(local_file_meta_stream(vec![filename])) + let schema_result = AvroFormat {} + .infer_schema(local_object_reader_stream(vec![filename])) .await; assert!(matches!( schema_result, diff --git a/datafusion/src/datasource/file_format/csv.rs b/datafusion/src/datasource/file_format/csv.rs index ea47223cc60a9..1abdca70ae2e0 100644 --- a/datafusion/src/datasource/file_format/csv.rs +++ b/datafusion/src/datasource/file_format/csv.rs @@ -26,7 +26,7 @@ use futures::StreamExt; use super::FileFormat; use super::PartitionedFile; -use crate::datasource::object_store::{FileMeta, FileMetaStream, ObjectStoreRegistry}; +use crate::datasource::object_store::{ObjectReader, ObjectReaderStream, ObjectStore}; use crate::error::Result; use crate::logical_plan::Expr; use crate::physical_plan::file_format::CsvExec; @@ -38,13 +38,11 @@ pub struct CsvFormat { has_header: bool, delimiter: u8, schema_infer_max_rec: Option, - object_store_registry: Arc, } impl Default for CsvFormat { fn default() -> Self { Self { - object_store_registry: Arc::new(ObjectStoreRegistry::new()), schema_infer_max_rec: None, has_header: true, delimiter: b',', @@ -53,14 +51,6 @@ impl Default for CsvFormat { } impl CsvFormat { - /// Create Parquet with the given object store and default values - pub fn new(object_store_registry: Arc) -> Self { - Self { - object_store_registry, - ..Default::default() - } - } - /// Set a limit in terms of records to scan to infer the schema /// - default to `None` (no limit) pub fn with_schema_infer_max_rec(&mut self, max_rec: Option) -> &mut Self { @@ -85,17 +75,13 @@ impl CsvFormat { #[async_trait] impl FileFormat for CsvFormat { - async fn infer_schema(&self, mut file_stream: FileMetaStream) -> Result { + async fn infer_schema(&self, mut readers: ObjectReaderStream) -> Result { let mut schemas = vec![]; + let mut records_to_read = self.schema_infer_max_rec.unwrap_or(std::usize::MAX); - while let Some(fmeta_res) = file_stream.next().await { - let fmeta = fmeta_res?; - let mut reader = self - .object_store_registry - .get_by_uri(&fmeta.path)? - .file_reader(fmeta)? - .sync_reader()?; + while let Some(obj_reader) = readers.next().await { + let mut reader = obj_reader?.sync_reader()?; let (schema, records_read) = arrow::csv::reader::infer_reader_schema( &mut reader, self.delimiter, @@ -116,12 +102,13 @@ impl FileFormat for CsvFormat { Ok(Arc::new(merged_schema)) } - async fn infer_stats(&self, _path: FileMeta) -> Result { + async fn infer_stats(&self, _reader: Arc) -> Result { Ok(Statistics::default()) } async fn create_physical_plan( &self, + object_store: Arc, schema: SchemaRef, files: Vec>, statistics: Statistics, @@ -131,7 +118,7 @@ impl FileFormat for CsvFormat { limit: Option, ) -> Result> { let exec = CsvExec::new( - Arc::clone(&self.object_store_registry), + object_store, // flattening this for now because CsvExec does not support partitioning yet files.into_iter().flatten().collect(), statistics, @@ -144,10 +131,6 @@ impl FileFormat for CsvFormat { ); Ok(Arc::new(exec)) } - - fn object_store_registry(&self) -> &Arc { - &self.object_store_registry - } } #[cfg(test)] @@ -156,7 +139,10 @@ mod tests { use super::*; use crate::{ - datasource::object_store::local::{local_file_meta, local_file_meta_stream}, + datasource::object_store::local::{ + local_file_meta, local_object_reader, local_object_reader_stream, + LocalFileSystem, + }, physical_plan::collect, }; @@ -266,18 +252,19 @@ mod tests { let filename = format!("{}/csv/{}", testdata, file_name); let format = CsvFormat::default(); let schema = format - .infer_schema(local_file_meta_stream(vec![filename.clone()])) + .infer_schema(local_object_reader_stream(vec![filename.clone()])) .await .expect("Schema inference"); let stats = format - .infer_stats(local_file_meta(filename.clone())) + .infer_stats(local_object_reader(filename.clone())) .await .expect("Stats inference"); let files = vec![vec![PartitionedFile { - file: local_file_meta(filename.to_owned()), + file_meta: local_file_meta(filename.to_owned()), }]]; let exec = format .create_physical_plan( + Arc::new(LocalFileSystem {}), schema, files, stats, diff --git a/datafusion/src/datasource/file_format/json.rs b/datafusion/src/datasource/file_format/json.rs index f7ed8517ae636..4e3ef37138df4 100644 --- a/datafusion/src/datasource/file_format/json.rs +++ b/datafusion/src/datasource/file_format/json.rs @@ -29,9 +29,7 @@ use futures::StreamExt; use super::FileFormat; use super::PartitionedFile; -use crate::datasource::object_store::FileMeta; -use crate::datasource::object_store::FileMetaStream; -use crate::datasource::object_store::ObjectStoreRegistry; +use crate::datasource::object_store::{ObjectReader, ObjectReaderStream, ObjectStore}; use crate::error::Result; use crate::logical_plan::Expr; use crate::physical_plan::file_format::NdJsonExec; @@ -41,27 +39,17 @@ use crate::physical_plan::Statistics; /// New line delimited JSON `FileFormat` implementation. pub struct JsonFormat { schema_infer_max_rec: Option, - object_store_registry: Arc, } impl Default for JsonFormat { fn default() -> Self { Self { schema_infer_max_rec: None, - object_store_registry: Arc::new(ObjectStoreRegistry::new()), } } } impl JsonFormat { - /// Create Parquet with the given object store and default values - pub fn new(object_store_registry: Arc) -> Self { - Self { - object_store_registry, - ..Default::default() - } - } - /// Set a limit in terms of records to scan to infer the schema /// - defaults to `None` (no limit) pub fn with_schema_infer_max_rec(&mut self, max_rec: Option) -> &mut Self { @@ -72,17 +60,11 @@ impl JsonFormat { #[async_trait] impl FileFormat for JsonFormat { - async fn infer_schema(&self, mut file_stream: FileMetaStream) -> Result { + async fn infer_schema(&self, mut readers: ObjectReaderStream) -> Result { let mut schemas = Vec::new(); let mut records_to_read = self.schema_infer_max_rec.unwrap_or(usize::MAX); - while let Some(fmeta_res) = file_stream.next().await { - let fmeta = fmeta_res?; - let reader = self - .object_store_registry - .get_by_uri(&fmeta.path)? - .file_reader(fmeta)? - .sync_reader()?; - let mut reader = BufReader::new(reader); + while let Some(obj_reader) = readers.next().await { + let mut reader = BufReader::new(obj_reader?.sync_reader()?); let iter = ValueIter::new(&mut reader, None); let schema = infer_json_schema_from_iterator(iter.take_while(|_| { let should_take = records_to_read > 0; @@ -99,12 +81,13 @@ impl FileFormat for JsonFormat { Ok(Arc::new(schema)) } - async fn infer_stats(&self, _path: FileMeta) -> Result { + async fn infer_stats(&self, _reader: Arc) -> Result { Ok(Statistics::default()) } async fn create_physical_plan( &self, + object_store: Arc, schema: SchemaRef, files: Vec>, statistics: Statistics, @@ -114,7 +97,7 @@ impl FileFormat for JsonFormat { limit: Option, ) -> Result> { let exec = NdJsonExec::new( - Arc::clone(&self.object_store_registry), + object_store, // flattening this for now because NdJsonExec does not support partitioning yet files.into_iter().flatten().collect(), statistics, @@ -125,10 +108,6 @@ impl FileFormat for JsonFormat { ); Ok(Arc::new(exec)) } - - fn object_store_registry(&self) -> &Arc { - &self.object_store_registry - } } #[cfg(test)] @@ -137,7 +116,10 @@ mod tests { use super::*; use crate::{ - datasource::object_store::local::{local_file_meta, local_file_meta_stream}, + datasource::object_store::local::{ + local_file_meta, local_object_reader, local_object_reader_stream, + LocalFileSystem, + }, physical_plan::collect, }; @@ -230,18 +212,19 @@ mod tests { let filename = "tests/jsons/2.json"; let format = JsonFormat::default(); let schema = format - .infer_schema(local_file_meta_stream(vec![filename.to_owned()])) + .infer_schema(local_object_reader_stream(vec![filename.to_owned()])) .await .expect("Schema inference"); let stats = format - .infer_stats(local_file_meta(filename.to_owned())) + .infer_stats(local_object_reader(filename.to_owned())) .await .expect("Stats inference"); let files = vec![vec![PartitionedFile { - file: local_file_meta(filename.to_owned()), + file_meta: local_file_meta(filename.to_owned()), }]]; let exec = format .create_physical_plan( + Arc::new(LocalFileSystem {}), schema, files, stats, diff --git a/datafusion/src/datasource/file_format/mod.rs b/datafusion/src/datasource/file_format/mod.rs index a876cac5e4e7f..d1ebc443e1223 100644 --- a/datafusion/src/datasource/file_format/mod.rs +++ b/datafusion/src/datasource/file_format/mod.rs @@ -34,18 +34,18 @@ use crate::physical_plan::{Accumulator, ExecutionPlan, Statistics}; use async_trait::async_trait; use futures::{Stream, StreamExt}; -use super::object_store::{FileMeta, FileMetaStream, ObjectStoreRegistry}; +use super::object_store::{FileMeta, ObjectReader, ObjectReaderStream, ObjectStore}; /// This trait abstracts all the file format specific implementations /// from the `TableProvider`. This helps code re-utilization accross /// providers that support the the same file formats. #[async_trait] pub trait FileFormat: Send + Sync { - /// Infer the the common schema of the files described by the path stream - async fn infer_schema(&self, paths: FileMetaStream) -> Result; + /// Infer the the common schema of the provided objects + async fn infer_schema(&self, readers: ObjectReaderStream) -> Result; - /// Infer the statistics for the file at the given path - async fn infer_stats(&self, path: FileMeta) -> Result; + /// Infer the statistics for the provided object + async fn infer_stats(&self, reader: Arc) -> Result; /// Take a list of files and convert it to the appropriate executor /// according to this file format. @@ -54,6 +54,7 @@ pub trait FileFormat: Send + Sync { #[allow(clippy::too_many_arguments)] async fn create_physical_plan( &self, + object_store: Arc, schema: SchemaRef, files: Vec>, statistics: Statistics, @@ -62,9 +63,6 @@ pub trait FileFormat: Send + Sync { filters: &[Expr], limit: Option, ) -> Result>; - - /// Get the oject store from which to read this file format - fn object_store_registry(&self) -> &Arc; } /// Get all files as well as the summary statistic @@ -159,7 +157,7 @@ pub async fn get_statistics_with_limit( /// TODO move back to crate::datasource::mod.rs once legacy cleaned up pub struct PartitionedFile { /// Path for the file (e.g. URL, filesystem path, etc) - pub file: FileMeta, + pub file_meta: FileMeta, // Values of partition columns to be appended to each row // pub partition_value: Option>, // We may include row group range here for a more fine-grained parallel execution @@ -171,7 +169,7 @@ pub type PartitionedFileStream = impl std::fmt::Display for PartitionedFile { fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { - write!(f, "{}", self.file) + write!(f, "{}", self.file_meta) } } diff --git a/datafusion/src/datasource/file_format/parquet.rs b/datafusion/src/datasource/file_format/parquet.rs index da132c5d76683..a49095f109ba9 100644 --- a/datafusion/src/datasource/file_format/parquet.rs +++ b/datafusion/src/datasource/file_format/parquet.rs @@ -37,10 +37,7 @@ use super::FileFormat; use super::PartitionedFile; use super::{create_max_min_accs, get_col_stats}; use crate::arrow::datatypes::{DataType, Field}; -use crate::datasource::object_store::FileMeta; -use crate::datasource::object_store::FileMetaStream; -use crate::datasource::object_store::ObjectReader; -use crate::datasource::object_store::ObjectStoreRegistry; +use crate::datasource::object_store::{ObjectReader, ObjectReaderStream, ObjectStore}; use crate::error::DataFusionError; use crate::error::Result; use crate::logical_plan::combine_filters; @@ -53,7 +50,6 @@ use crate::scalar::ScalarValue; /// The Apache Parquet `FileFormat` implementation pub struct ParquetFormat { - object_store_registry: Arc, enable_pruning: bool, } @@ -61,20 +57,11 @@ impl Default for ParquetFormat { fn default() -> Self { Self { enable_pruning: true, - object_store_registry: Arc::new(ObjectStoreRegistry::new()), } } } impl ParquetFormat { - /// Create Parquet with the given object store and default values - pub fn new(object_store_registry: Arc) -> Self { - Self { - object_store_registry, - ..Default::default() - } - } - /// Activate statistics based row group level pruning /// - defaults to true pub fn with_enable_pruning(&mut self, enable: bool) -> &mut Self { @@ -85,25 +72,26 @@ impl ParquetFormat { #[async_trait] impl FileFormat for ParquetFormat { - async fn infer_schema(&self, mut paths: FileMetaStream) -> Result { + async fn infer_schema(&self, mut readers: ObjectReaderStream) -> Result { // We currently get the schema information from the first file rather than do // schema merging and this is a limitation. // See https://issues.apache.org/jira/browse/ARROW-11017 - let first_file = paths + let first_file = readers .next() .await .ok_or_else(|| DataFusionError::Plan("No data file found".to_owned()))??; - let (schema, _) = fetch_metadata(&self.object_store_registry, first_file)?; + let (schema, _) = fetch_metadata(first_file)?; Ok(Arc::new(schema)) } - async fn infer_stats(&self, path: FileMeta) -> Result { - let (_, stats) = fetch_metadata(&self.object_store_registry, path)?; + async fn infer_stats(&self, reader: Arc) -> Result { + let (_, stats) = fetch_metadata(reader)?; Ok(stats) } async fn create_physical_plan( &self, + object_store: Arc, schema: SchemaRef, files: Vec>, statistics: Statistics, @@ -122,7 +110,7 @@ impl FileFormat for ParquetFormat { }; Ok(Arc::new(ParquetExec::new( - Arc::clone(&self.object_store_registry), + object_store, files, statistics, schema, @@ -132,10 +120,6 @@ impl FileFormat for ParquetFormat { limit, ))) } - - fn object_store_registry(&self) -> &Arc { - &self.object_store_registry - } } fn summarize_min_max( @@ -261,12 +245,8 @@ fn summarize_min_max( } /// Read and parse the metadata of the Parquet file at location `path` -fn fetch_metadata( - object_store_registry: &ObjectStoreRegistry, - fmeta: FileMeta, -) -> Result<(Schema, Statistics)> { - let object_store = object_store_registry.get_by_uri(&fmeta.path)?; - let obj_reader = ChunkObjectReader(object_store.file_reader(fmeta)?); +fn fetch_metadata(object_reader: Arc) -> Result<(Schema, Statistics)> { + let obj_reader = ChunkObjectReader(object_reader); let file_reader = Arc::new(SerializedFileReader::new(obj_reader)?); let mut arrow_reader = ParquetFileArrowReader::new(file_reader); let schema = arrow_reader.get_schema()?; @@ -345,7 +325,10 @@ impl ChunkReader for ChunkObjectReader { #[cfg(test)] mod tests { use crate::{ - datasource::object_store::local::{local_file_meta, local_file_meta_stream}, + datasource::object_store::local::{ + local_file_meta, local_object_reader, local_object_reader_stream, + LocalFileSystem, + }, physical_plan::collect, }; @@ -605,18 +588,19 @@ mod tests { let filename = format!("{}/{}", testdata, file_name); let format = ParquetFormat::default(); let schema = format - .infer_schema(local_file_meta_stream(vec![filename.clone()])) + .infer_schema(local_object_reader_stream(vec![filename.clone()])) .await .expect("Schema inference"); let stats = format - .infer_stats(local_file_meta(filename.clone())) + .infer_stats(local_object_reader(filename.clone())) .await .expect("Stats inference"); let files = vec![vec![PartitionedFile { - file: local_file_meta(filename.clone()), + file_meta: local_file_meta(filename.clone()), }]]; let exec = format .create_physical_plan( + Arc::new(LocalFileSystem {}), schema, files, stats, diff --git a/datafusion/src/datasource/listing.rs b/datafusion/src/datasource/listing.rs index ff28d3b214701..5e9da121ac73c 100644 --- a/datafusion/src/datasource/listing.rs +++ b/datafusion/src/datasource/listing.rs @@ -34,7 +34,7 @@ use crate::{ use super::{ datasource::TableProviderFilterPushDown, file_format::{FileFormat, PartitionedFileStream}, - object_store::ObjectStoreRegistry, + object_store::{ObjectStore, ObjectStoreRegistry}, TableProvider, }; @@ -53,7 +53,7 @@ pub struct ListingOptions { /// Note that only `DataType::Utf8` is supported for the column type. /// TODO implement case where partitions.len() > 0 pub partitions: Vec, - /// Set true to try to guess statistics from the file parse it. + /// Set true to try to guess statistics from the files. /// This can add a lot of overhead as it requires files to /// be opened and partially parsed. pub collect_stat: bool, @@ -63,18 +63,23 @@ pub struct ListingOptions { } impl ListingOptions { - /// Infer the schema of the files at the given path, including the partitioning + /// Infer the schema of the files at the given uri, including the partitioning /// columns. /// /// This method will not be called by the table itself but before creating it. /// This way when creating the logical plan we can decide to resolve the schema /// locally or ask a remote service to do it (e.g a scheduler). - pub async fn infer_schema(&self, path: &str) -> Result { - let object_store = self.format.object_store_registry().get_by_uri(path)?; + pub async fn infer_schema( + &self, + object_store_registry: Arc, + uri: &str, + ) -> Result { + let object_store = object_store_registry.get_by_uri(uri)?; let file_stream = object_store - .list_file_with_suffix(path, &self.file_extension) - .await?; - let file_schema = self.format.infer_schema(file_stream).await?; + .list_file_with_suffix(uri, &self.file_extension) + .await? + .map(move |file_meta| object_store.file_reader(file_meta?.sized_file)); + let file_schema = self.format.infer_schema(Box::pin(file_stream)).await?; // Add the partition columns to the file schema let mut fields = file_schema.fields().clone(); for part in &self.partitions { @@ -87,7 +92,9 @@ impl ListingOptions { /// An implementation of `TableProvider` that uses the object store /// or file system listing capability to get the list of files. pub struct ListingTable { - path: String, + // TODO pass object_store_registry to scan() instead + object_store_registry: Arc, + uri: String, schema: SchemaRef, options: ListingOptions, } @@ -95,14 +102,17 @@ pub struct ListingTable { impl ListingTable { /// Create new table that lists the FS to get the files to scan. pub fn new( - path: impl Into, + // TODO pass object_store_registry to scan() instead + object_store_registry: Arc, + uri: impl Into, // the schema must be resolved before creating the table schema: SchemaRef, options: ListingOptions, ) -> Self { - let path: String = path.into(); + let uri: String = uri.into(); Self { - path, + object_store_registry, + uri, schema, options, } @@ -126,12 +136,16 @@ impl TableProvider for ListingTable { filters: &[Expr], limit: Option, ) -> Result> { - let (partitioned_file_lists, statistics) = - self.list_files_for_scan(filters, limit).await?; + // TODO object_store_registry should be provided as param here + let object_store = self.object_store_registry.get_by_uri(&self.uri)?; + let (partitioned_file_lists, statistics) = self + .list_files_for_scan(Arc::clone(&object_store), filters.to_vec(), limit) + .await?; // create the execution plan self.options .format .create_physical_plan( + object_store, self.schema(), partitioned_file_lists, statistics, @@ -154,31 +168,35 @@ impl TableProvider for ListingTable { impl ListingTable { async fn list_files_for_scan( &self, - filters: &[Expr], + object_store: Arc, + // `Vec` required here for lifetime reasons + filters: Vec, limit: Option, ) -> Result<(Vec>, Statistics)> { // list files (with partitions) let file_list = pruned_partition_list( - self.options.format.object_store_registry(), - &self.path, - filters, + object_store.as_ref(), + &self.uri, + &filters, &self.options.file_extension, &self.options.partitions, ) .await?; // collect the statistics if required by the config - let files = file_list.then(|part_file| async { - let part_file = part_file?; - let statistics = if self.options.collect_stat { - self.options - .format - .infer_stats(part_file.file.clone()) - .await? - } else { - Statistics::default() - }; - Ok((part_file, statistics)) as Result<(PartitionedFile, Statistics)> + let files = file_list.then(move |part_file| { + let object_store = object_store.clone(); + async move { + let part_file = part_file?; + let statistics = if self.options.collect_stat { + let object_reader = object_store + .file_reader(part_file.file_meta.sized_file.clone())?; + self.options.format.infer_stats(object_reader).await? + } else { + Statistics::default() + }; + Ok((part_file, statistics)) as Result<(PartitionedFile, Statistics)> + } }); let (files, statistics) = @@ -187,7 +205,7 @@ impl ListingTable { if files.is_empty() { return Err(DataFusionError::Plan(format!( "No files found at {} with file extension {}", - self.path, self.options.file_extension, + self.uri, self.options.file_extension, ))); } @@ -198,7 +216,7 @@ impl ListingTable { /// Discover the partitions on the given path and prune out files /// relative to irrelevant partitions using `filters` expressions async fn pruned_partition_list( - registry: &ObjectStoreRegistry, + store: &dyn ObjectStore, path: &str, _filters: &[Expr], file_extension: &str, @@ -206,11 +224,10 @@ async fn pruned_partition_list( ) -> Result { if partition_names.is_empty() { Ok(Box::pin( - registry - .get_by_uri(path)? + store .list_file_with_suffix(path, file_extension) .await? - .map(|f| Ok(PartitionedFile { file: f? })), + .map(|f| Ok(PartitionedFile { file_meta: f? })), )) } else { todo!("use filters to prune partitions") @@ -233,9 +250,16 @@ fn split_files( #[cfg(test)] mod tests { + use std::io::Read; + + use futures::AsyncRead; + use crate::datasource::{ file_format::{avro::AvroFormat, parquet::ParquetFormat}, - object_store::{FileMeta, FileMetaStream, ListEntryStream, ObjectStore}, + object_store::{ + FileMeta, FileMetaStream, ListEntryStream, ObjectReader, ObjectStore, + SizedFile, + }, }; use super::*; @@ -243,9 +267,11 @@ mod tests { #[test] fn test_split_files() { let new_partitioned_file = |path: &str| PartitionedFile { - file: FileMeta { - path: path.to_owned(), - size: 10, + file_meta: FileMeta { + sized_file: SizedFile { + path: path.to_owned(), + size: 10, + }, last_modified: None, }, }; @@ -321,9 +347,13 @@ mod tests { max_partitions: 2, collect_stat: true, }; + let object_store_reg = Arc::new(ObjectStoreRegistry::new()); // here we resolve the schema locally - let schema = opt.infer_schema(&filename).await.expect("Infer schema"); - let table = ListingTable::new(&filename, schema, opt); + let schema = opt + .infer_schema(Arc::clone(&object_store_reg), &filename) + .await + .expect("Infer schema"); + let table = ListingTable::new(object_store_reg, &filename, schema, opt); Ok(Arc::new(table)) } @@ -333,12 +363,11 @@ mod tests { output_partitioning: usize, ) -> Result<()> { let registry = ObjectStoreRegistry::new(); - registry.register_store( - "mock".to_owned(), - Arc::new(MockObjectStore { files_in_folder }), - ); + let mock_store: Arc = + Arc::new(MockObjectStore { files_in_folder }); + registry.register_store("mock".to_owned(), Arc::clone(&mock_store)); - let format = AvroFormat::new(Arc::new(registry)); + let format = AvroFormat {}; let opt = ListingOptions { file_extension: "".to_owned(), @@ -348,11 +377,18 @@ mod tests { collect_stat: true, }; + let object_store_reg = Arc::new(ObjectStoreRegistry::new()); + let schema = Schema::new(vec![Field::new("a", DataType::Boolean, false)]); - let table = ListingTable::new("mock://bucket/key-prefix", Arc::new(schema), opt); + let table = ListingTable::new( + object_store_reg, + "mock://bucket/key-prefix", + Arc::new(schema), + opt, + ); - let (file_list, _) = table.list_files_for_scan(&[], None).await?; + let (file_list, _) = table.list_files_for_scan(mock_store, vec![], None).await?; assert_eq!(file_list.len(), output_partitioning); @@ -370,8 +406,10 @@ mod tests { let prefix = prefix.to_owned(); let files = (0..self.files_in_folder).map(move |i| { Ok(FileMeta { - path: format!("{}file{}", prefix, i), - size: 100, + sized_file: SizedFile { + path: format!("{}file{}", prefix, i), + size: 100, + }, last_modified: None, }) }); @@ -386,10 +424,32 @@ mod tests { unimplemented!() } - fn file_reader( + fn file_reader(&self, _file: SizedFile) -> Result> { + Ok(Arc::new(MockObjectReader {})) + } + } + + struct MockObjectReader {} + + #[async_trait] + impl ObjectReader for MockObjectReader { + async fn chunk_reader( + &self, + _start: u64, + _length: usize, + ) -> Result> { + unimplemented!() + } + + fn sync_chunk_reader( &self, - _file: FileMeta, - ) -> Result> { + _start: u64, + _length: usize, + ) -> Result> { + unimplemented!() + } + + fn length(&self) -> u64 { unimplemented!() } } diff --git a/datafusion/src/datasource/object_store/local.rs b/datafusion/src/datasource/object_store/local.rs index 2a331d8e6f154..c18af1cd8f448 100644 --- a/datafusion/src/datasource/object_store/local.rs +++ b/datafusion/src/datasource/object_store/local.rs @@ -30,6 +30,8 @@ use crate::datasource::object_store::{ use crate::error::DataFusionError; use crate::error::Result; +use super::{ObjectReaderStream, SizedFile}; + #[derive(Debug)] /// Local File System as Object Store. pub struct LocalFileSystem; @@ -48,17 +50,17 @@ impl ObjectStore for LocalFileSystem { todo!() } - fn file_reader(&self, file: FileMeta) -> Result> { + fn file_reader(&self, file: SizedFile) -> Result> { Ok(Arc::new(LocalFileReader::new(file)?)) } } struct LocalFileReader { - file: FileMeta, + file: SizedFile, } impl LocalFileReader { - fn new(file: FileMeta) -> Result { + fn new(file: SizedFile) -> Result { Ok(Self { file }) } } @@ -95,8 +97,10 @@ impl ObjectReader for LocalFileReader { async fn list_all(prefix: String) -> Result { fn get_meta(path: String, metadata: Metadata) -> FileMeta { FileMeta { - path, - size: metadata.len(), + sized_file: SizedFile { + path, + size: metadata.len(), + }, last_modified: metadata.modified().map(chrono::DateTime::from).ok(), } } @@ -148,17 +152,26 @@ async fn list_all(prefix: String) -> Result { } } -/// Create a stream of `FileMeta` applying `local_file_meta` to each path in `files` -pub fn local_file_meta_stream(files: Vec) -> FileMetaStream { - Box::pin(futures::stream::iter(files).map(|f| Ok(local_file_meta(f)))) +/// Create a stream of `ObjectReader` by opening each file in the `files` vector +pub fn local_object_reader_stream(files: Vec) -> ObjectReaderStream { + Box::pin(futures::stream::iter(files).map(|f| Ok(local_object_reader(f)))) } -/// Helper method to fetch the file size at given path and create a `FileMeta` +/// Helper method to convert a file location to an ObjectReader +pub fn local_object_reader(file: String) -> Arc { + LocalFileSystem + .file_reader(local_file_meta(file).sized_file) + .expect("File not found") +} + +/// Helper method to fetch the file size and date at given path and create a `FileMeta` pub fn local_file_meta(file: String) -> FileMeta { let metadata = fs::metadata(&file).expect("Local file metadata"); FileMeta { - size: metadata.len(), - path: file, + sized_file: SizedFile { + size: metadata.len(), + path: file, + }, last_modified: metadata.modified().map(chrono::DateTime::from).ok(), } } @@ -193,8 +206,8 @@ mod tests { let mut files = list_all(tmp.path().to_str().unwrap().to_string()).await?; while let Some(file) = files.next().await { let file = file?; - assert_eq!(file.size, 0); - all_files.insert(file.path); + assert_eq!(file.size(), 0); + all_files.insert(file.path().to_owned()); } assert_eq!(all_files.len(), 3); diff --git a/datafusion/src/datasource/object_store/mod.rs b/datafusion/src/datasource/object_store/mod.rs index 6b932f1c4ee3d..eb4ede2e4618d 100644 --- a/datafusion/src/datasource/object_store/mod.rs +++ b/datafusion/src/datasource/object_store/mod.rs @@ -33,11 +33,12 @@ use local::LocalFileSystem; use crate::error::{DataFusionError, Result}; -/// Object Reader for one file in an object store +/// Object Reader for one file in an object store. +/// /// Note that the dynamic dispatch on the reader might /// have some performance impacts. #[async_trait] -pub trait ObjectReader { +pub trait ObjectReader: Send + Sync { /// Get reader for a part [start, start + length] in the file asynchronously async fn chunk_reader(&self, start: u64, length: usize) -> Result>; @@ -68,13 +69,23 @@ pub enum ListEntry { Prefix(String), } -/// Complete file path with size we got from object store +/// The path and size of the file. #[derive(Debug, Clone)] -pub struct FileMeta { - /// Path of the file +pub struct SizedFile { + /// Path of the file. It is relative to the current object + /// store (it does not specify the xx:// scheme). pub path: String, /// File size in total pub size: u64, +} + +/// Description of a file as returned by the listing command of a +/// given object store. The resulting path is relative to the +/// object store that generated it. +#[derive(Debug, Clone)] +pub struct FileMeta { + /// The path and size of the file. + pub sized_file: SizedFile, /// The last modification time of the file according to the /// object store metadata. This information might be used by /// catalog systems like Delta Lake for time travel (see @@ -82,9 +93,22 @@ pub struct FileMeta { pub last_modified: Option>, } +impl FileMeta { + /// The path that describes this file. It is relative to the + /// associated object store. + pub fn path(&self) -> &str { + &self.sized_file.path + } + + /// The size of the file. + pub fn size(&self) -> u64 { + self.sized_file.size + } +} + impl std::fmt::Display for FileMeta { fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { - write!(f, "{} (size: {})", self.path, self.size) + write!(f, "{} (size: {})", self.path(), self.size()) } } @@ -96,6 +120,10 @@ pub type FileMetaStream = pub type ListEntryStream = Pin> + Send + Sync + 'static>>; +/// Stream readers opened on a given object store +pub type ObjectReaderStream = + Pin>> + Send + Sync + 'static>>; + /// A ObjectStore abstracts access to an underlying file/object storage. /// It maps strings (e.g. URLs, filesystem paths, etc) to sources of bytes #[async_trait] @@ -113,7 +141,7 @@ pub trait ObjectStore: Sync + Send + Debug { let suffix = suffix.to_owned(); Ok(Box::pin(file_stream.filter(move |fr| { let has_suffix = match fr { - Ok(f) => f.path.ends_with(&suffix), + Ok(f) => f.path().ends_with(&suffix), Err(_) => true, }; async move { has_suffix } @@ -129,7 +157,7 @@ pub trait ObjectStore: Sync + Send + Debug { ) -> Result; /// Get object reader for one file - fn file_reader(&self, file: FileMeta) -> Result>; + fn file_reader(&self, file: SizedFile) -> Result>; } static LOCAL_SCHEME: &str = "file"; diff --git a/datafusion/src/physical_plan/file_format/avro.rs b/datafusion/src/physical_plan/file_format/avro.rs index d079748ec85f7..23d912803250e 100644 --- a/datafusion/src/physical_plan/file_format/avro.rs +++ b/datafusion/src/physical_plan/file_format/avro.rs @@ -17,7 +17,7 @@ //! Execution plan for reading line-delimited Avro files use crate::datasource::file_format::PartitionedFile; -use crate::datasource::object_store::ObjectStoreRegistry; +use crate::datasource::object_store::ObjectStore; use crate::error::{DataFusionError, Result}; #[cfg(feature = "avro")] use crate::physical_plan::RecordBatchStream; @@ -42,7 +42,7 @@ use std::{ /// Execution plan for scanning Avro data source #[derive(Debug, Clone)] pub struct AvroExec { - object_store_registry: Arc, + object_store: Arc, files: Vec, statistics: Statistics, schema: SchemaRef, @@ -56,7 +56,7 @@ impl AvroExec { /// Create a new JSON reader execution plan provided file list and schema /// TODO: support partitiond file list (Vec>) pub fn new( - object_store_registry: Arc, + object_store: Arc, files: Vec, statistics: Statistics, schema: SchemaRef, @@ -72,7 +72,7 @@ impl AvroExec { }; Self { - object_store_registry, + object_store, files, statistics, schema, @@ -126,9 +126,8 @@ impl ExecutionPlan for AvroExec { #[cfg(feature = "avro")] async fn execute(&self, partition: usize) -> Result { let file = self - .object_store_registry - .get_by_uri(&self.files[partition].file.path)? - .file_reader(self.files[partition].file.clone())? + .object_store + .file_reader(self.files[partition].file_meta.sized_file.clone())? .sync_reader()?; let proj = self.projection.as_ref().map(|p| { @@ -162,7 +161,7 @@ impl ExecutionPlan for AvroExec { self.limit, self.files .iter() - .map(|f| f.file.path.as_str()) + .map(|f| f.file_meta.path()) .collect::>() .join(", ") ) @@ -244,7 +243,7 @@ impl RecordBatchStream for AvroStream<'_, R> { mod tests { use crate::datasource::object_store::local::{ - local_file_meta, local_file_meta_stream, + local_file_meta, local_object_reader_stream, LocalFileSystem, }; use super::*; @@ -258,13 +257,13 @@ mod tests { let testdata = crate::test_util::arrow_test_data(); let filename = format!("{}/avro/alltypes_plain.avro", testdata); let avro_exec = AvroExec::new( - Arc::new(ObjectStoreRegistry::new()), + Arc::new(LocalFileSystem {}), vec![PartitionedFile { - file: local_file_meta(filename.clone()), + file_meta: local_file_meta(filename.clone()), }], Statistics::default(), - AvroFormat::default() - .infer_schema(local_file_meta_stream(vec![filename])) + AvroFormat {} + .infer_schema(local_object_reader_stream(vec![filename])) .await?, Some(vec![0, 1, 2]), 1024, diff --git a/datafusion/src/physical_plan/file_format/csv.rs b/datafusion/src/physical_plan/file_format/csv.rs index e662f7cecbec9..df6022cc44aab 100644 --- a/datafusion/src/physical_plan/file_format/csv.rs +++ b/datafusion/src/physical_plan/file_format/csv.rs @@ -18,7 +18,7 @@ //! Execution plan for reading CSV files use crate::datasource::file_format::PartitionedFile; -use crate::datasource::object_store::ObjectStoreRegistry; +use crate::datasource::object_store::ObjectStore; use crate::error::{DataFusionError, Result}; use crate::physical_plan::{ DisplayFormatType, ExecutionPlan, Partitioning, RecordBatchStream, @@ -40,7 +40,7 @@ use async_trait::async_trait; /// Execution plan for scanning a CSV file #[derive(Debug, Clone)] pub struct CsvExec { - object_store_registry: Arc, + object_store: Arc, /// List of data files files: Vec, /// Schema representing the CSV file @@ -66,7 +66,7 @@ impl CsvExec { /// TODO: support partitiond file list (Vec>) #[allow(clippy::too_many_arguments)] pub fn new( - object_store_registry: Arc, + object_store: Arc, files: Vec, statistics: Statistics, schema: SchemaRef, @@ -84,7 +84,7 @@ impl CsvExec { }; Self { - object_store_registry, + object_store, files, schema, statistics, @@ -136,9 +136,8 @@ impl ExecutionPlan for CsvExec { async fn execute(&self, partition: usize) -> Result { let file = self - .object_store_registry - .get_by_uri(&self.files[partition].file.path)? - .file_reader(self.files[partition].file.clone())? + .object_store + .file_reader(self.files[partition].file_meta.sized_file.clone())? .sync_reader()?; Ok(Box::pin(CsvStream::try_new_from_reader( @@ -167,7 +166,7 @@ impl ExecutionPlan for CsvExec { self.limit, self.files .iter() - .map(|f| f.file.path.as_str()) + .map(|f| f.file_meta.path()) .collect::>() .join(", ") ) @@ -236,7 +235,8 @@ impl RecordBatchStream for CsvStream { mod tests { use super::*; use crate::{ - datasource::object_store::local::local_file_meta, test::aggr_test_schema, + datasource::object_store::local::{local_file_meta, LocalFileSystem}, + test::aggr_test_schema, }; use futures::StreamExt; @@ -247,9 +247,9 @@ mod tests { let filename = "aggregate_test_100.csv"; let path = format!("{}/csv/{}", testdata, filename); let csv = CsvExec::new( - Arc::new(ObjectStoreRegistry::new()), + Arc::new(LocalFileSystem {}), vec![PartitionedFile { - file: local_file_meta(path), + file_meta: local_file_meta(path), }], Statistics::default(), schema, @@ -280,9 +280,9 @@ mod tests { let filename = "aggregate_test_100.csv"; let path = format!("{}/csv/{}", testdata, filename); let csv = CsvExec::new( - Arc::new(ObjectStoreRegistry::new()), + Arc::new(LocalFileSystem {}), vec![PartitionedFile { - file: local_file_meta(path), + file_meta: local_file_meta(path), }], Statistics::default(), schema, diff --git a/datafusion/src/physical_plan/file_format/json.rs b/datafusion/src/physical_plan/file_format/json.rs index 4e62c5e298405..090fd3f59ef35 100644 --- a/datafusion/src/physical_plan/file_format/json.rs +++ b/datafusion/src/physical_plan/file_format/json.rs @@ -20,7 +20,7 @@ use async_trait::async_trait; use futures::Stream; use crate::datasource::file_format::PartitionedFile; -use crate::datasource::object_store::ObjectStoreRegistry; +use crate::datasource::object_store::ObjectStore; use crate::error::{DataFusionError, Result}; use crate::physical_plan::{ DisplayFormatType, ExecutionPlan, Partitioning, RecordBatchStream, @@ -43,7 +43,7 @@ use std::{ /// Execution plan for scanning NdJson data source #[derive(Debug, Clone)] pub struct NdJsonExec { - object_store_registry: Arc, + object_store: Arc, files: Vec, statistics: Statistics, schema: SchemaRef, @@ -57,7 +57,7 @@ impl NdJsonExec { /// Create a new JSON reader execution plan provided file list and schema /// TODO: support partitiond file list (Vec>) pub fn new( - object_store_registry: Arc, + object_store: Arc, files: Vec, statistics: Statistics, schema: SchemaRef, @@ -73,7 +73,7 @@ impl NdJsonExec { }; Self { - object_store_registry, + object_store, files, statistics, schema, @@ -126,9 +126,8 @@ impl ExecutionPlan for NdJsonExec { }); let file = self - .object_store_registry - .get_by_uri(&self.files[partition].file.path)? - .file_reader(self.files[partition].file.clone())? + .object_store + .file_reader(self.files[partition].file_meta.sized_file.clone())? .sync_reader()?; let json_reader = json::Reader::new(file, self.schema(), self.batch_size, proj); @@ -150,7 +149,7 @@ impl ExecutionPlan for NdJsonExec { self.limit, self.files .iter() - .map(|f| f.file.path.as_str()) + .map(|f| f.file_meta.path()) .collect::>() .join(", ") ) @@ -229,7 +228,9 @@ mod tests { use crate::datasource::{ file_format::{json::JsonFormat, FileFormat}, - object_store::local::{local_file_meta, local_file_meta_stream}, + object_store::local::{ + local_file_meta, local_object_reader_stream, LocalFileSystem, + }, }; use super::*; @@ -238,7 +239,7 @@ mod tests { async fn infer_schema(path: String) -> Result { JsonFormat::default() - .infer_schema(local_file_meta_stream(vec![path])) + .infer_schema(local_object_reader_stream(vec![path])) .await } @@ -247,9 +248,9 @@ mod tests { use arrow::datatypes::DataType; let path = format!("{}/1.json", TEST_DATA_BASE); let exec = NdJsonExec::new( - Arc::new(ObjectStoreRegistry::new()), + Arc::new(LocalFileSystem {}), vec![PartitionedFile { - file: local_file_meta(path.clone()), + file_meta: local_file_meta(path.clone()), }], Default::default(), infer_schema(path).await?, @@ -302,9 +303,9 @@ mod tests { async fn nd_json_exec_file_projection() -> Result<()> { let path = format!("{}/1.json", TEST_DATA_BASE); let exec = NdJsonExec::new( - Arc::new(ObjectStoreRegistry::new()), + Arc::new(LocalFileSystem {}), vec![PartitionedFile { - file: local_file_meta(path.clone()), + file_meta: local_file_meta(path.clone()), }], Default::default(), infer_schema(path).await?, diff --git a/datafusion/src/physical_plan/file_format/parquet.rs b/datafusion/src/physical_plan/file_format/parquet.rs index 1be3d8b3284a3..3d73d0550fa08 100644 --- a/datafusion/src/physical_plan/file_format/parquet.rs +++ b/datafusion/src/physical_plan/file_format/parquet.rs @@ -22,7 +22,7 @@ use std::sync::Arc; use std::{any::Any, convert::TryInto}; use crate::datasource::file_format::parquet::ChunkObjectReader; -use crate::datasource::object_store::ObjectStoreRegistry; +use crate::datasource::object_store::ObjectStore; use crate::{ error::{DataFusionError, Result}, logical_plan::{Column, Expr}, @@ -64,7 +64,7 @@ use crate::datasource::file_format::{FilePartition, PartitionedFile}; /// Execution plan for scanning one or more Parquet partitions #[derive(Debug, Clone)] pub struct ParquetExec { - object_store_registry: Arc, + object_store: Arc, /// Parquet partitions to read partitions: Vec, /// Schema after projection is applied @@ -114,7 +114,7 @@ impl ParquetExec { /// Even if `limit` is set, ParquetExec rounds up the number of records to the next `batch_size`. #[allow(clippy::too_many_arguments)] pub fn new( - object_store_registry: Arc, + object_store: Arc, files: Vec>, statistics: Statistics, schema: SchemaRef, @@ -161,7 +161,7 @@ impl ParquetExec { Self::project(&projection, schema, statistics); Self { - object_store_registry, + object_store, partitions, schema: projected_schema, projection, @@ -289,11 +289,11 @@ impl ExecutionPlan for ParquetExec { let predicate_builder = self.predicate_builder.clone(); let batch_size = self.batch_size; let limit = self.limit; - let object_store_registry = Arc::clone(&self.object_store_registry); + let object_store = Arc::clone(&self.object_store); task::spawn_blocking(move || { if let Err(e) = read_partition( - &object_store_registry, + object_store.as_ref(), partition_index, partition, metrics, @@ -470,7 +470,7 @@ fn build_row_group_predicate( #[allow(clippy::too_many_arguments)] fn read_partition( - object_store_registry: &ObjectStoreRegistry, + object_store: &dyn ObjectStore, partition_index: usize, partition: ParquetPartition, metrics: ExecutionPlanMetricsSet, @@ -485,12 +485,11 @@ fn read_partition( 'outer: for partitioned_file in all_files { let file_metrics = ParquetFileMetrics::new( partition_index, - &*partitioned_file.file.path, + &*partitioned_file.file_meta.path(), &metrics, ); - let object_reader = object_store_registry - .get_by_uri(&partitioned_file.file.path)? - .file_reader(partitioned_file.file.clone())?; + let object_reader = + object_store.file_reader(partitioned_file.file_meta.sized_file.clone())?; let mut file_reader = SerializedFileReader::new(ChunkObjectReader(object_reader))?; if let Some(predicate_builder) = predicate_builder { @@ -543,7 +542,9 @@ fn read_partition( mod tests { use crate::datasource::{ file_format::{parquet::ParquetFormat, FileFormat}, - object_store::local::{local_file_meta, local_file_meta_stream}, + object_store::local::{ + local_file_meta, local_object_reader_stream, LocalFileSystem, + }, }; use super::*; @@ -560,13 +561,13 @@ mod tests { let testdata = crate::test_util::parquet_test_data(); let filename = format!("{}/alltypes_plain.parquet", testdata); let parquet_exec = ParquetExec::new( - Arc::new(ObjectStoreRegistry::new()), + Arc::new(LocalFileSystem {}), vec![vec![PartitionedFile { - file: local_file_meta(filename.clone()), + file_meta: local_file_meta(filename.clone()), }]], Statistics::default(), ParquetFormat::default() - .infer_schema(local_file_meta_stream(vec![filename])) + .infer_schema(local_object_reader_stream(vec![filename])) .await?, Some(vec![0, 1, 2]), None, From fe19d62183c1998cce35a5f85f6351e6ff34dbb3 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 5 Oct 2021 16:54:01 +0200 Subject: [PATCH 16/17] [refacto] grouped params to avoid too_many_arguments --- datafusion/src/datasource/file_format/avro.rs | 52 ++++++++---------- datafusion/src/datasource/file_format/csv.rs | 52 ++++++++---------- datafusion/src/datasource/file_format/json.rs | 51 ++++++++---------- datafusion/src/datasource/file_format/mod.rs | 41 +++++++++----- .../src/datasource/file_format/parquet.rs | 53 +++++++++---------- datafusion/src/datasource/listing.rs | 27 +++++----- 6 files changed, 134 insertions(+), 142 deletions(-) diff --git a/datafusion/src/datasource/file_format/avro.rs b/datafusion/src/datasource/file_format/avro.rs index cfe15090e23ff..1ca538867ccab 100644 --- a/datafusion/src/datasource/file_format/avro.rs +++ b/datafusion/src/datasource/file_format/avro.rs @@ -24,12 +24,10 @@ use arrow::{self, datatypes::SchemaRef}; use async_trait::async_trait; use futures::StreamExt; -use super::FileFormat; -use super::PartitionedFile; +use super::{FileFormat, PhysicalPlanConfig}; use crate::avro_to_arrow::read_avro_schema_from_reader; -use crate::datasource::object_store::{ObjectReader, ObjectReaderStream, ObjectStore}; +use crate::datasource::object_store::{ObjectReader, ObjectReaderStream}; use crate::error::Result; -use crate::logical_plan::Expr; use crate::physical_plan::file_format::AvroExec; use crate::physical_plan::ExecutionPlan; use crate::physical_plan::Statistics; @@ -56,24 +54,17 @@ impl FileFormat for AvroFormat { async fn create_physical_plan( &self, - object_store: Arc, - schema: SchemaRef, - files: Vec>, - statistics: Statistics, - projection: &Option>, - batch_size: usize, - _filters: &[Expr], - limit: Option, + conf: PhysicalPlanConfig, ) -> Result> { let exec = AvroExec::new( - object_store, + conf.object_store, // flattening this for now because CsvExec does not support partitioning yet - files.into_iter().flatten().collect(), - statistics, - schema, - projection.clone(), - batch_size, - limit, + conf.files.into_iter().flatten().collect(), + conf.statistics, + conf.schema, + conf.projection, + conf.batch_size, + conf.limit, ); Ok(Arc::new(exec)) } @@ -83,9 +74,12 @@ impl FileFormat for AvroFormat { #[cfg(feature = "avro")] mod tests { use crate::{ - datasource::object_store::local::{ - local_file_meta, local_object_reader, local_object_reader_stream, - LocalFileSystem, + datasource::{ + file_format::PartitionedFile, + object_store::local::{ + local_file_meta, local_object_reader, local_object_reader_stream, + LocalFileSystem, + }, }, physical_plan::collect, }; @@ -354,7 +348,7 @@ mod tests { .infer_schema(local_object_reader_stream(vec![filename.clone()])) .await .expect("Schema inference"); - let stats = format + let statistics = format .infer_stats(local_object_reader(filename.clone())) .await .expect("Stats inference"); @@ -362,16 +356,16 @@ mod tests { file_meta: local_file_meta(filename.to_owned()), }]]; let exec = format - .create_physical_plan( - Arc::new(LocalFileSystem {}), + .create_physical_plan(PhysicalPlanConfig { + object_store: Arc::new(LocalFileSystem {}), schema, files, - stats, - projection, + statistics, + projection: projection.clone(), batch_size, - &[], + filters: vec![], limit, - ) + }) .await?; Ok(exec) } diff --git a/datafusion/src/datasource/file_format/csv.rs b/datafusion/src/datasource/file_format/csv.rs index 1abdca70ae2e0..d619227e43060 100644 --- a/datafusion/src/datasource/file_format/csv.rs +++ b/datafusion/src/datasource/file_format/csv.rs @@ -24,11 +24,9 @@ use arrow::{self, datatypes::SchemaRef}; use async_trait::async_trait; use futures::StreamExt; -use super::FileFormat; -use super::PartitionedFile; -use crate::datasource::object_store::{ObjectReader, ObjectReaderStream, ObjectStore}; +use super::{FileFormat, PhysicalPlanConfig}; +use crate::datasource::object_store::{ObjectReader, ObjectReaderStream}; use crate::error::Result; -use crate::logical_plan::Expr; use crate::physical_plan::file_format::CsvExec; use crate::physical_plan::ExecutionPlan; use crate::physical_plan::Statistics; @@ -108,26 +106,19 @@ impl FileFormat for CsvFormat { async fn create_physical_plan( &self, - object_store: Arc, - schema: SchemaRef, - files: Vec>, - statistics: Statistics, - projection: &Option>, - batch_size: usize, - _filters: &[Expr], - limit: Option, + conf: PhysicalPlanConfig, ) -> Result> { let exec = CsvExec::new( - object_store, + conf.object_store, // flattening this for now because CsvExec does not support partitioning yet - files.into_iter().flatten().collect(), - statistics, - schema, + conf.files.into_iter().flatten().collect(), + conf.statistics, + conf.schema, self.has_header, self.delimiter, - projection.clone(), - batch_size, - limit, + conf.projection, + conf.batch_size, + conf.limit, ); Ok(Arc::new(exec)) } @@ -139,9 +130,12 @@ mod tests { use super::*; use crate::{ - datasource::object_store::local::{ - local_file_meta, local_object_reader, local_object_reader_stream, - LocalFileSystem, + datasource::{ + file_format::{PartitionedFile, PhysicalPlanConfig}, + object_store::local::{ + local_file_meta, local_object_reader, local_object_reader_stream, + LocalFileSystem, + }, }, physical_plan::collect, }; @@ -255,7 +249,7 @@ mod tests { .infer_schema(local_object_reader_stream(vec![filename.clone()])) .await .expect("Schema inference"); - let stats = format + let statistics = format .infer_stats(local_object_reader(filename.clone())) .await .expect("Stats inference"); @@ -263,16 +257,16 @@ mod tests { file_meta: local_file_meta(filename.to_owned()), }]]; let exec = format - .create_physical_plan( - Arc::new(LocalFileSystem {}), + .create_physical_plan(PhysicalPlanConfig { + object_store: Arc::new(LocalFileSystem {}), schema, files, - stats, - projection, + statistics, + projection: projection.clone(), batch_size, - &[], + filters: vec![], limit, - ) + }) .await?; Ok(exec) } diff --git a/datafusion/src/datasource/file_format/json.rs b/datafusion/src/datasource/file_format/json.rs index 4e3ef37138df4..fd7f6580ab8a4 100644 --- a/datafusion/src/datasource/file_format/json.rs +++ b/datafusion/src/datasource/file_format/json.rs @@ -28,10 +28,9 @@ use async_trait::async_trait; use futures::StreamExt; use super::FileFormat; -use super::PartitionedFile; -use crate::datasource::object_store::{ObjectReader, ObjectReaderStream, ObjectStore}; +use super::PhysicalPlanConfig; +use crate::datasource::object_store::{ObjectReader, ObjectReaderStream}; use crate::error::Result; -use crate::logical_plan::Expr; use crate::physical_plan::file_format::NdJsonExec; use crate::physical_plan::ExecutionPlan; use crate::physical_plan::Statistics; @@ -87,24 +86,17 @@ impl FileFormat for JsonFormat { async fn create_physical_plan( &self, - object_store: Arc, - schema: SchemaRef, - files: Vec>, - statistics: Statistics, - projection: &Option>, - batch_size: usize, - _filters: &[Expr], - limit: Option, + conf: PhysicalPlanConfig, ) -> Result> { let exec = NdJsonExec::new( - object_store, + conf.object_store, // flattening this for now because NdJsonExec does not support partitioning yet - files.into_iter().flatten().collect(), - statistics, - schema, - projection.clone(), - batch_size, - limit, + conf.files.into_iter().flatten().collect(), + conf.statistics, + conf.schema, + conf.projection, + conf.batch_size, + conf.limit, ); Ok(Arc::new(exec)) } @@ -116,9 +108,12 @@ mod tests { use super::*; use crate::{ - datasource::object_store::local::{ - local_file_meta, local_object_reader, local_object_reader_stream, - LocalFileSystem, + datasource::{ + file_format::{PartitionedFile, PhysicalPlanConfig}, + object_store::local::{ + local_file_meta, local_object_reader, local_object_reader_stream, + LocalFileSystem, + }, }, physical_plan::collect, }; @@ -215,7 +210,7 @@ mod tests { .infer_schema(local_object_reader_stream(vec![filename.to_owned()])) .await .expect("Schema inference"); - let stats = format + let statistics = format .infer_stats(local_object_reader(filename.to_owned())) .await .expect("Stats inference"); @@ -223,16 +218,16 @@ mod tests { file_meta: local_file_meta(filename.to_owned()), }]]; let exec = format - .create_physical_plan( - Arc::new(LocalFileSystem {}), + .create_physical_plan(PhysicalPlanConfig { + object_store: Arc::new(LocalFileSystem {}), schema, files, - stats, - projection, + statistics, + projection: projection.clone(), batch_size, - &[], + filters: vec![], limit, - ) + }) .await?; Ok(exec) } diff --git a/datafusion/src/datasource/file_format/mod.rs b/datafusion/src/datasource/file_format/mod.rs index d1ebc443e1223..b315c99b7d140 100644 --- a/datafusion/src/datasource/file_format/mod.rs +++ b/datafusion/src/datasource/file_format/mod.rs @@ -36,32 +36,47 @@ use futures::{Stream, StreamExt}; use super::object_store::{FileMeta, ObjectReader, ObjectReaderStream, ObjectStore}; +/// The configurations to be passed when creating a physical plan for +/// a given file format. +pub struct PhysicalPlanConfig { + /// Store from which the `files` should be fetched + pub object_store: Arc, + /// Schema before projection + pub schema: SchemaRef, + /// Partitioned fields to process in the executor + pub files: Vec>, + /// Estimated overall statistics of source plan + pub statistics: Statistics, + /// Columns on which to project the data + pub projection: Option>, + /// The maximum number of records per arrow column + pub batch_size: usize, + /// The filters that where pushed down to this execution plan + pub filters: Vec, + /// The minimum number of records required from this source plan + pub limit: Option, +} + /// This trait abstracts all the file format specific implementations /// from the `TableProvider`. This helps code re-utilization accross /// providers that support the the same file formats. #[async_trait] pub trait FileFormat: Send + Sync { - /// Infer the the common schema of the provided objects + /// Infer the common schema of the provided objects. The objects will usually + /// be analysed up to a given number of records or files (as specified in the + /// format config) then give the estimated common schema. This might fail if + /// the files have schemas that cannot be merged. async fn infer_schema(&self, readers: ObjectReaderStream) -> Result; - /// Infer the statistics for the provided object + /// Infer the statistics for the provided object. The cost and accuracy of the + /// estimated statistics might vary greatly between file formats. async fn infer_stats(&self, reader: Arc) -> Result; /// Take a list of files and convert it to the appropriate executor /// according to this file format. - /// TODO group params into TableDescription(schema,files,stats) and - /// ScanOptions(projection,batch_size,filters) to avoid too_many_arguments - #[allow(clippy::too_many_arguments)] async fn create_physical_plan( &self, - object_store: Arc, - schema: SchemaRef, - files: Vec>, - statistics: Statistics, - projection: &Option>, - batch_size: usize, - filters: &[Expr], - limit: Option, + conf: PhysicalPlanConfig, ) -> Result>; } diff --git a/datafusion/src/datasource/file_format/parquet.rs b/datafusion/src/datasource/file_format/parquet.rs index a49095f109ba9..9e5d623b7bb15 100644 --- a/datafusion/src/datasource/file_format/parquet.rs +++ b/datafusion/src/datasource/file_format/parquet.rs @@ -34,14 +34,13 @@ use parquet::file::serialized_reader::SerializedFileReader; use parquet::file::statistics::Statistics as ParquetStatistics; use super::FileFormat; -use super::PartitionedFile; +use super::PhysicalPlanConfig; use super::{create_max_min_accs, get_col_stats}; use crate::arrow::datatypes::{DataType, Field}; -use crate::datasource::object_store::{ObjectReader, ObjectReaderStream, ObjectStore}; +use crate::datasource::object_store::{ObjectReader, ObjectReaderStream}; use crate::error::DataFusionError; use crate::error::Result; use crate::logical_plan::combine_filters; -use crate::logical_plan::Expr; use crate::physical_plan::expressions::{MaxAccumulator, MinAccumulator}; use crate::physical_plan::file_format::ParquetExec; use crate::physical_plan::ExecutionPlan; @@ -91,33 +90,26 @@ impl FileFormat for ParquetFormat { async fn create_physical_plan( &self, - object_store: Arc, - schema: SchemaRef, - files: Vec>, - statistics: Statistics, - projection: &Option>, - batch_size: usize, - filters: &[Expr], - limit: Option, + conf: PhysicalPlanConfig, ) -> Result> { // If enable pruning then combine the filters to build the predicate. // If disable pruning then set the predicate to None, thus readers // will not prune data based on the statistics. let predicate = if self.enable_pruning { - combine_filters(filters) + combine_filters(&conf.filters) } else { None }; Ok(Arc::new(ParquetExec::new( - object_store, - files, - statistics, - schema, - projection.clone(), + conf.object_store, + conf.files, + conf.statistics, + conf.schema, + conf.projection, predicate, - batch_size, - limit, + conf.batch_size, + conf.limit, ))) } } @@ -325,9 +317,12 @@ impl ChunkReader for ChunkObjectReader { #[cfg(test)] mod tests { use crate::{ - datasource::object_store::local::{ - local_file_meta, local_object_reader, local_object_reader_stream, - LocalFileSystem, + datasource::{ + file_format::PartitionedFile, + object_store::local::{ + local_file_meta, local_object_reader, local_object_reader_stream, + LocalFileSystem, + }, }, physical_plan::collect, }; @@ -591,7 +586,7 @@ mod tests { .infer_schema(local_object_reader_stream(vec![filename.clone()])) .await .expect("Schema inference"); - let stats = format + let statistics = format .infer_stats(local_object_reader(filename.clone())) .await .expect("Stats inference"); @@ -599,16 +594,16 @@ mod tests { file_meta: local_file_meta(filename.clone()), }]]; let exec = format - .create_physical_plan( - Arc::new(LocalFileSystem {}), + .create_physical_plan(PhysicalPlanConfig { + object_store: Arc::new(LocalFileSystem {}), schema, files, - stats, - projection, + statistics, + projection: projection.clone(), batch_size, - &[], + filters: vec![], limit, - ) + }) .await?; Ok(exec) } diff --git a/datafusion/src/datasource/listing.rs b/datafusion/src/datasource/listing.rs index 5e9da121ac73c..b1370bec3efb4 100644 --- a/datafusion/src/datasource/listing.rs +++ b/datafusion/src/datasource/listing.rs @@ -33,7 +33,7 @@ use crate::{ use super::{ datasource::TableProviderFilterPushDown, - file_format::{FileFormat, PartitionedFileStream}, + file_format::{FileFormat, PartitionedFileStream, PhysicalPlanConfig}, object_store::{ObjectStore, ObjectStoreRegistry}, TableProvider, }; @@ -139,21 +139,21 @@ impl TableProvider for ListingTable { // TODO object_store_registry should be provided as param here let object_store = self.object_store_registry.get_by_uri(&self.uri)?; let (partitioned_file_lists, statistics) = self - .list_files_for_scan(Arc::clone(&object_store), filters.to_vec(), limit) + .list_files_for_scan(Arc::clone(&object_store), filters, limit) .await?; // create the execution plan self.options .format - .create_physical_plan( + .create_physical_plan(PhysicalPlanConfig { object_store, - self.schema(), - partitioned_file_lists, + schema: self.schema(), + files: partitioned_file_lists, statistics, - projection, + projection: projection.clone(), batch_size, - filters, + filters: filters.to_vec(), limit, - ) + }) .await } @@ -166,18 +166,17 @@ impl TableProvider for ListingTable { } impl ListingTable { - async fn list_files_for_scan( - &self, + async fn list_files_for_scan<'a>( + &'a self, object_store: Arc, - // `Vec` required here for lifetime reasons - filters: Vec, + filters: &'a [Expr], limit: Option, ) -> Result<(Vec>, Statistics)> { // list files (with partitions) let file_list = pruned_partition_list( object_store.as_ref(), &self.uri, - &filters, + filters, &self.options.file_extension, &self.options.partitions, ) @@ -388,7 +387,7 @@ mod tests { opt, ); - let (file_list, _) = table.list_files_for_scan(mock_store, vec![], None).await?; + let (file_list, _) = table.list_files_for_scan(mock_store, &[], None).await?; assert_eq!(file_list.len(), output_partitioning); From 8e8fd981e8a79b0080efaa8f5e68ef8dd3975680 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 5 Oct 2021 17:19:55 +0200 Subject: [PATCH 17/17] [fix] get_by_uri also returns path --- datafusion/src/datasource/listing.rs | 15 ++++++++------ datafusion/src/datasource/object_store/mod.rs | 20 ++++++++++++------- 2 files changed, 22 insertions(+), 13 deletions(-) diff --git a/datafusion/src/datasource/listing.rs b/datafusion/src/datasource/listing.rs index b1370bec3efb4..f4b8f7755166d 100644 --- a/datafusion/src/datasource/listing.rs +++ b/datafusion/src/datasource/listing.rs @@ -74,9 +74,9 @@ impl ListingOptions { object_store_registry: Arc, uri: &str, ) -> Result { - let object_store = object_store_registry.get_by_uri(uri)?; + let (object_store, path) = object_store_registry.get_by_uri(uri)?; let file_stream = object_store - .list_file_with_suffix(uri, &self.file_extension) + .list_file_with_suffix(path, &self.file_extension) .await? .map(move |file_meta| object_store.file_reader(file_meta?.sized_file)); let file_schema = self.format.infer_schema(Box::pin(file_stream)).await?; @@ -137,9 +137,9 @@ impl TableProvider for ListingTable { limit: Option, ) -> Result> { // TODO object_store_registry should be provided as param here - let object_store = self.object_store_registry.get_by_uri(&self.uri)?; + let (object_store, path) = self.object_store_registry.get_by_uri(&self.uri)?; let (partitioned_file_lists, statistics) = self - .list_files_for_scan(Arc::clone(&object_store), filters, limit) + .list_files_for_scan(Arc::clone(&object_store), path, filters, limit) .await?; // create the execution plan self.options @@ -169,13 +169,14 @@ impl ListingTable { async fn list_files_for_scan<'a>( &'a self, object_store: Arc, + path: &'a str, filters: &'a [Expr], limit: Option, ) -> Result<(Vec>, Statistics)> { // list files (with partitions) let file_list = pruned_partition_list( object_store.as_ref(), - &self.uri, + path, filters, &self.options.file_extension, &self.options.partitions, @@ -387,7 +388,9 @@ mod tests { opt, ); - let (file_list, _) = table.list_files_for_scan(mock_store, &[], None).await?; + let (file_list, _) = table + .list_files_for_scan(mock_store, "bucket/key-prefix", &[], None) + .await?; assert_eq!(file_list.len(), output_partitioning); diff --git a/datafusion/src/datasource/object_store/mod.rs b/datafusion/src/datasource/object_store/mod.rs index eb4ede2e4618d..b16684a9db0aa 100644 --- a/datafusion/src/datasource/object_store/mod.rs +++ b/datafusion/src/datasource/object_store/mod.rs @@ -216,12 +216,17 @@ impl ObjectStoreRegistry { } /// Get a suitable store for the URI based on it's scheme. For example: - /// URI with scheme file or no schema will return the default LocalFS store, - /// URI with scheme s3 will return the S3 store if it's registered. - pub fn get_by_uri(&self, uri: &str) -> Result> { - if let Some((scheme, _)) = uri.split_once(':') { + /// - URI with scheme `file://` or no schema will return the default LocalFS store + /// - URI with scheme `s3://` will return the S3 store if it's registered + /// Returns a tuple with the store and the path of the file in that store + /// (URI=scheme://path). + pub fn get_by_uri<'a>( + &self, + uri: &'a str, + ) -> Result<(Arc, &'a str)> { + if let Some((scheme, path)) = uri.split_once("://") { let stores = self.object_stores.read().unwrap(); - stores + let store = stores .get(&*scheme.to_lowercase()) .map(Clone::clone) .ok_or_else(|| { @@ -229,9 +234,10 @@ impl ObjectStoreRegistry { "No suitable object store found for {}", scheme )) - }) + })?; + Ok((store, path)) } else { - Ok(Arc::new(LocalFileSystem)) + Ok((Arc::new(LocalFileSystem), uri)) } } }