From 5e11154bb7882257918563b22d4328f0e0f79f13 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 12 Oct 2021 19:24:15 +0200 Subject: [PATCH 01/39] [feat] stubs for provider re-organization --- ballista/rust/core/proto/ballista.proto | 26 ++ ballista/rust/scheduler/src/lib.rs | 11 + datafusion/src/datasource/format/mod.rs | 151 ++++++ datafusion/src/datasource/format/parquet.rs | 484 ++++++++++++++++++++ datafusion/src/datasource/listing.rs | 294 ++++++++++++ datafusion/src/datasource/mod.rs | 2 + datafusion/src/physical_plan/parquet.rs | 51 +++ 7 files changed, 1019 insertions(+) create mode 100644 datafusion/src/datasource/format/mod.rs create mode 100644 datafusion/src/datasource/format/parquet.rs create mode 100644 datafusion/src/datasource/listing.rs diff --git a/ballista/rust/core/proto/ballista.proto b/ballista/rust/core/proto/ballista.proto index 9a2ec710411b..a8d1dd15c639 100644 --- a/ballista/rust/core/proto/ballista.proto +++ b/ballista/rust/core/proto/ballista.proto @@ -949,6 +949,30 @@ message GetFileMetadataParams { FileType file_type = 2; } +message ParquetConfig { + // fields of datasource::listing::FormatOptions::Parquet +} + +message ListingConfig { + string extension = 1; + oneof format { + ParquetConfig parquet = 2; + // csv, json, ... + } +} + +message GetSchemaParams { + string path = 1; + oneof provider_config { + ListingConfig listing = 2; + // delta, iceberg... (maybe replace oneof with something more exensible) + } +} + +message GetSchemaResult { + Schema schema = 1; +} + message GetFileMetadataResult { Schema schema = 1; repeated FilePartitionMetadata partitions = 2; @@ -962,6 +986,8 @@ service SchedulerGrpc { // Executors must poll the scheduler for heartbeat and to receive tasks rpc PollWork (PollWorkParams) returns (PollWorkResult) {} + rpc GetSchema (GetSchemaParams) returns (GetSchemaResult) {} + rpc GetFileMetadata (GetFileMetadataParams) returns (GetFileMetadataResult) {} rpc ExecuteQuery (ExecuteQueryParams) returns (ExecuteQueryResult) {} diff --git a/ballista/rust/scheduler/src/lib.rs b/ballista/rust/scheduler/src/lib.rs index 47caf4c21ede..0a06cd7b4995 100644 --- a/ballista/rust/scheduler/src/lib.rs +++ b/ballista/rust/scheduler/src/lib.rs @@ -22,6 +22,7 @@ pub mod planner; #[cfg(feature = "sled")] mod standalone; pub mod state; +use ballista_core::serde::protobuf::{GetSchemaParams, GetSchemaResult}; #[cfg(feature = "sled")] pub use standalone::new_standalone_scheduler; @@ -268,6 +269,16 @@ impl SchedulerGrpc for SchedulerServer { } } + /// this service would replace get_file_meatadata + async fn get_schema( + &self, + _request: Request, + ) -> std::result::Result, tonic::Status> { + // if GetSchemaParams contains a provider config of type ListingConfig + // use ListingOptions.infer_schema() + todo!() + } + async fn get_file_metadata( &self, request: Request, diff --git a/datafusion/src/datasource/format/mod.rs b/datafusion/src/datasource/format/mod.rs new file mode 100644 index 000000000000..33c0717899de --- /dev/null +++ b/datafusion/src/datasource/format/mod.rs @@ -0,0 +1,151 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! Module containing helper methods for the various file formats + +// pub mod csv; +// pub mod json; +pub mod parquet; + +use crate::arrow::datatypes::{Schema, SchemaRef}; +use crate::physical_plan::expressions::{MaxAccumulator, MinAccumulator}; +use crate::physical_plan::{Accumulator, ColumnStatistics, Statistics}; + +use super::PartitionedFile; + +/// Get all files as well as the summary statistic +/// if the optional `limit` is provided, includes only sufficient files +/// needed to read up to `limit` number of rows +pub fn get_statistics_with_limit( + all_files: &[PartitionedFile], + schema: SchemaRef, + limit: Option, +) -> (Vec, Statistics) { + let mut all_files = all_files.to_vec(); + + let mut total_byte_size = 0; + let mut null_counts = vec![0; schema.fields().len()]; + let mut has_statistics = false; + 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 in &all_files { + num_files += 1; + let file_stats = &file.statistics; + 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); + if let Some(vec) = &file_stats.column_statistics { + has_statistics = true; + for (i, cs) in vec.iter().enumerate() { + null_counts[i] += cs.null_count.unwrap_or(0); + + if let Some(max_value) = &mut max_values[i] { + if let Some(file_max) = cs.max_value.clone() { + match max_value.update(&[file_max]) { + Ok(_) => {} + Err(_) => { + max_values[i] = None; + } + } + } + } + + if let Some(min_value) = &mut min_values[i] { + if let Some(file_min) = cs.min_value.clone() { + match min_value.update(&[file_min]) { + Ok(_) => {} + Err(_) => { + min_values[i] = None; + } + } + } + } + } + } + if num_rows > limit.unwrap_or(usize::MAX) { + break; + } + } + if num_files < all_files.len() { + is_exact = false; + all_files.truncate(num_files); + } + + let column_stats = if has_statistics { + Some(get_col_stats( + &*schema, + null_counts, + &mut max_values, + &mut min_values, + )) + } else { + None + }; + + let statistics = Statistics { + num_rows: Some(num_rows as usize), + total_byte_size: Some(total_byte_size as usize), + column_statistics: column_stats, + is_exact, + }; + (all_files, statistics) +} + +fn create_max_min_accs( + schema: &Schema, +) -> (Vec>, Vec>) { + let max_values: Vec> = schema + .fields() + .iter() + .map(|field| MaxAccumulator::try_new(field.data_type()).ok()) + .collect::>(); + let min_values: Vec> = schema + .fields() + .iter() + .map(|field| MinAccumulator::try_new(field.data_type()).ok()) + .collect::>(); + (max_values, min_values) +} + +fn get_col_stats( + schema: &Schema, + null_counts: Vec, + max_values: &mut Vec>, + min_values: &mut Vec>, +) -> Vec { + (0..schema.fields().len()) + .map(|i| { + let max_value = match &max_values[i] { + Some(max_value) => max_value.evaluate().ok(), + None => None, + }; + let min_value = match &min_values[i] { + Some(min_value) => min_value.evaluate().ok(), + None => None, + }; + ColumnStatistics { + null_count: Some(null_counts[i] as usize), + max_value, + min_value, + distinct_count: None, + } + }) + .collect() +} diff --git a/datafusion/src/datasource/format/parquet.rs b/datafusion/src/datasource/format/parquet.rs new file mode 100644 index 000000000000..cb2faae3870d --- /dev/null +++ b/datafusion/src/datasource/format/parquet.rs @@ -0,0 +1,484 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! Parquet format helper methods + +use std::fs::File; +use std::sync::Arc; + +use arrow::datatypes::Schema; +use parquet::arrow::ArrowReader; +use parquet::arrow::ParquetFileArrowReader; +use parquet::file::serialized_reader::SerializedFileReader; +use parquet::file::statistics::Statistics as ParquetStatistics; + +use super::{create_max_min_accs, get_col_stats}; +use crate::arrow::datatypes::{DataType, Field}; +use crate::error::Result; +use crate::physical_plan::expressions::{MaxAccumulator, MinAccumulator}; +use crate::physical_plan::{Accumulator, Statistics}; +use crate::scalar::ScalarValue; + +fn summarize_min_max( + max_values: &mut Vec>, + min_values: &mut Vec>, + fields: &[Field], + i: usize, + stat: &ParquetStatistics, +) { + match stat { + ParquetStatistics::Boolean(s) => { + if let DataType::Boolean = fields[i].data_type() { + if s.has_min_max_set() { + if let Some(max_value) = &mut max_values[i] { + match max_value.update(&[ScalarValue::Boolean(Some(*s.max()))]) { + Ok(_) => {} + Err(_) => { + max_values[i] = None; + } + } + } + if let Some(min_value) = &mut min_values[i] { + match min_value.update(&[ScalarValue::Boolean(Some(*s.min()))]) { + Ok(_) => {} + Err(_) => { + min_values[i] = None; + } + } + } + } + } + } + ParquetStatistics::Int32(s) => { + if let DataType::Int32 = fields[i].data_type() { + if s.has_min_max_set() { + if let Some(max_value) = &mut max_values[i] { + match max_value.update(&[ScalarValue::Int32(Some(*s.max()))]) { + Ok(_) => {} + Err(_) => { + max_values[i] = None; + } + } + } + if let Some(min_value) = &mut min_values[i] { + match min_value.update(&[ScalarValue::Int32(Some(*s.min()))]) { + Ok(_) => {} + Err(_) => { + min_values[i] = None; + } + } + } + } + } + } + ParquetStatistics::Int64(s) => { + if let DataType::Int64 = fields[i].data_type() { + if s.has_min_max_set() { + if let Some(max_value) = &mut max_values[i] { + match max_value.update(&[ScalarValue::Int64(Some(*s.max()))]) { + Ok(_) => {} + Err(_) => { + max_values[i] = None; + } + } + } + if let Some(min_value) = &mut min_values[i] { + match min_value.update(&[ScalarValue::Int64(Some(*s.min()))]) { + Ok(_) => {} + Err(_) => { + min_values[i] = None; + } + } + } + } + } + } + ParquetStatistics::Float(s) => { + if let DataType::Float32 = fields[i].data_type() { + if s.has_min_max_set() { + if let Some(max_value) = &mut max_values[i] { + match max_value.update(&[ScalarValue::Float32(Some(*s.max()))]) { + Ok(_) => {} + Err(_) => { + max_values[i] = None; + } + } + } + if let Some(min_value) = &mut min_values[i] { + match min_value.update(&[ScalarValue::Float32(Some(*s.min()))]) { + Ok(_) => {} + Err(_) => { + min_values[i] = None; + } + } + } + } + } + } + ParquetStatistics::Double(s) => { + if let DataType::Float64 = fields[i].data_type() { + if s.has_min_max_set() { + if let Some(max_value) = &mut max_values[i] { + match max_value.update(&[ScalarValue::Float64(Some(*s.max()))]) { + Ok(_) => {} + Err(_) => { + max_values[i] = None; + } + } + } + if let Some(min_value) = &mut min_values[i] { + match min_value.update(&[ScalarValue::Float64(Some(*s.min()))]) { + Ok(_) => {} + Err(_) => { + min_values[i] = None; + } + } + } + } + } + } + _ => {} + } +} + +/// Read and parse the metadata of the Parquet file at location `path` +pub fn fetch_metadata(path: &str) -> Result<(Schema, Statistics)> { + let file = File::open(path)?; + let file_reader = Arc::new(SerializedFileReader::new(file)?); + let mut arrow_reader = ParquetFileArrowReader::new(file_reader); + let schema = arrow_reader.get_schema()?; + let num_fields = schema.fields().len(); + let fields = schema.fields().to_vec(); + let meta_data = arrow_reader.get_metadata(); + + let mut num_rows = 0; + let mut total_byte_size = 0; + let mut null_counts = vec![0; num_fields]; + let mut has_statistics = false; + + let (mut max_values, mut min_values) = create_max_min_accs(&schema); + + for row_group_meta in meta_data.row_groups() { + num_rows += row_group_meta.num_rows(); + total_byte_size += row_group_meta.total_byte_size(); + + let columns_null_counts = row_group_meta + .columns() + .iter() + .flat_map(|c| c.statistics().map(|stats| stats.null_count())); + + for (i, cnt) in columns_null_counts.enumerate() { + null_counts[i] += cnt as usize + } + + for (i, column) in row_group_meta.columns().iter().enumerate() { + if let Some(stat) = column.statistics() { + has_statistics = true; + summarize_min_max(&mut max_values, &mut min_values, &fields, i, stat) + } + } + } + + let column_stats = if has_statistics { + Some(get_col_stats( + &schema, + null_counts, + &mut max_values, + &mut min_values, + )) + } else { + None + }; + + let statistics = Statistics { + num_rows: Some(num_rows as usize), + total_byte_size: Some(total_byte_size as usize), + column_statistics: column_stats, + is_exact: true, + }; + + Ok((schema, statistics)) +} + +// #[cfg(test)] +// mod tests { +// use super::*; +// use arrow::array::{ +// BinaryArray, BooleanArray, Float32Array, Float64Array, Int32Array, +// TimestampNanosecondArray, +// }; +// use arrow::record_batch::RecordBatch; +// use futures::StreamExt; + +// #[tokio::test] +// async fn read_small_batches() -> Result<()> { +// let table = load_table("alltypes_plain.parquet")?; +// let projection = None; +// let exec = table.scan(&projection, 2, &[], None)?; +// let stream = exec.execute(0).await?; + +// let _ = stream +// .map(|batch| { +// let batch = batch.unwrap(); +// assert_eq!(11, batch.num_columns()); +// assert_eq!(2, batch.num_rows()); +// }) +// .fold(0, |acc, _| async move { acc + 1i32 }) +// .await; + +// // test metadata +// assert_eq!(exec.statistics().num_rows, Some(8)); +// assert_eq!(exec.statistics().total_byte_size, Some(671)); + +// Ok(()) +// } + +// #[tokio::test] +// async fn read_alltypes_plain_parquet() -> Result<()> { +// let table = load_table("alltypes_plain.parquet")?; + +// let x: Vec = table +// .schema() +// .fields() +// .iter() +// .map(|f| format!("{}: {:?}", f.name(), f.data_type())) +// .collect(); +// let y = x.join("\n"); +// assert_eq!( +// "id: Int32\n\ +// bool_col: Boolean\n\ +// tinyint_col: Int32\n\ +// smallint_col: Int32\n\ +// int_col: Int32\n\ +// bigint_col: Int64\n\ +// float_col: Float32\n\ +// double_col: Float64\n\ +// date_string_col: Binary\n\ +// string_col: Binary\n\ +// timestamp_col: Timestamp(Nanosecond, None)", +// y +// ); + +// let projection = None; +// let batch = get_first_batch(table, &projection).await?; + +// assert_eq!(11, batch.num_columns()); +// assert_eq!(8, batch.num_rows()); + +// Ok(()) +// } + +// #[tokio::test] +// async fn read_bool_alltypes_plain_parquet() -> Result<()> { +// let table = load_table("alltypes_plain.parquet")?; +// let projection = Some(vec![1]); +// let batch = get_first_batch(table, &projection).await?; + +// assert_eq!(1, batch.num_columns()); +// assert_eq!(8, batch.num_rows()); + +// let array = batch +// .column(0) +// .as_any() +// .downcast_ref::() +// .unwrap(); +// let mut values: Vec = vec![]; +// for i in 0..batch.num_rows() { +// values.push(array.value(i)); +// } + +// assert_eq!( +// "[true, false, true, false, true, false, true, false]", +// format!("{:?}", values) +// ); + +// Ok(()) +// } + +// #[tokio::test] +// async fn read_i32_alltypes_plain_parquet() -> Result<()> { +// let table = load_table("alltypes_plain.parquet")?; +// let projection = Some(vec![0]); +// let batch = get_first_batch(table, &projection).await?; + +// assert_eq!(1, batch.num_columns()); +// assert_eq!(8, batch.num_rows()); + +// let array = batch +// .column(0) +// .as_any() +// .downcast_ref::() +// .unwrap(); +// let mut values: Vec = vec![]; +// for i in 0..batch.num_rows() { +// values.push(array.value(i)); +// } + +// assert_eq!("[4, 5, 6, 7, 2, 3, 0, 1]", format!("{:?}", values)); + +// Ok(()) +// } + +// #[tokio::test] +// async fn read_i96_alltypes_plain_parquet() -> Result<()> { +// let table = load_table("alltypes_plain.parquet")?; +// let projection = Some(vec![10]); +// let batch = get_first_batch(table, &projection).await?; + +// assert_eq!(1, batch.num_columns()); +// assert_eq!(8, batch.num_rows()); + +// let array = batch +// .column(0) +// .as_any() +// .downcast_ref::() +// .unwrap(); +// let mut values: Vec = vec![]; +// for i in 0..batch.num_rows() { +// values.push(array.value(i)); +// } + +// assert_eq!("[1235865600000000000, 1235865660000000000, 1238544000000000000, 1238544060000000000, 1233446400000000000, 1233446460000000000, 1230768000000000000, 1230768060000000000]", format!("{:?}", values)); + +// Ok(()) +// } + +// #[tokio::test] +// async fn read_f32_alltypes_plain_parquet() -> Result<()> { +// let table = load_table("alltypes_plain.parquet")?; +// let projection = Some(vec![6]); +// let batch = get_first_batch(table, &projection).await?; + +// assert_eq!(1, batch.num_columns()); +// assert_eq!(8, batch.num_rows()); + +// let array = batch +// .column(0) +// .as_any() +// .downcast_ref::() +// .unwrap(); +// let mut values: Vec = vec![]; +// for i in 0..batch.num_rows() { +// values.push(array.value(i)); +// } + +// assert_eq!( +// "[0.0, 1.1, 0.0, 1.1, 0.0, 1.1, 0.0, 1.1]", +// format!("{:?}", values) +// ); + +// Ok(()) +// } + +// #[tokio::test] +// async fn read_f64_alltypes_plain_parquet() -> Result<()> { +// let table = load_table("alltypes_plain.parquet")?; +// let projection = Some(vec![7]); +// let batch = get_first_batch(table, &projection).await?; + +// assert_eq!(1, batch.num_columns()); +// assert_eq!(8, batch.num_rows()); + +// let array = batch +// .column(0) +// .as_any() +// .downcast_ref::() +// .unwrap(); +// let mut values: Vec = vec![]; +// for i in 0..batch.num_rows() { +// values.push(array.value(i)); +// } + +// assert_eq!( +// "[0.0, 10.1, 0.0, 10.1, 0.0, 10.1, 0.0, 10.1]", +// format!("{:?}", values) +// ); + +// Ok(()) +// } + +// #[tokio::test] +// async fn read_binary_alltypes_plain_parquet() -> Result<()> { +// let table = load_table("alltypes_plain.parquet")?; +// let projection = Some(vec![9]); +// let batch = get_first_batch(table, &projection).await?; + +// assert_eq!(1, batch.num_columns()); +// assert_eq!(8, batch.num_rows()); + +// let array = batch +// .column(0) +// .as_any() +// .downcast_ref::() +// .unwrap(); +// let mut values: Vec<&str> = vec![]; +// for i in 0..batch.num_rows() { +// values.push(std::str::from_utf8(array.value(i)).unwrap()); +// } + +// assert_eq!( +// "[\"0\", \"1\", \"0\", \"1\", \"0\", \"1\", \"0\", \"1\"]", +// format!("{:?}", values) +// ); + +// Ok(()) +// } + +// fn load_table(name: &str) -> Result> { +// let testdata = crate::test_util::parquet_test_data(); +// let filename = format!("{}/{}", testdata, name); +// let table = ParquetTable::try_new(&filename, 2)?; +// Ok(Arc::new(table)) +// } + +// async fn get_first_batch( +// table: Arc, +// projection: &Option>, +// ) -> Result { +// let exec = table.scan(projection, 1024, &[], None)?; +// let mut it = exec.execute(0).await?; +// it.next() +// .await +// .expect("should have received at least one batch") +// .map_err(|e| e.into()) +// } + +// #[test] +// fn combine_zero_filters() { +// let result = combine_filters(&[]); +// assert_eq!(result, None); +// } + +// #[test] +// fn combine_one_filter() { +// use crate::logical_plan::{binary_expr, col, lit, Operator}; +// let filter = binary_expr(col("c1"), Operator::Lt, lit(1)); +// let result = combine_filters(&[filter.clone()]); +// assert_eq!(result, Some(filter)); +// } + +// #[test] +// fn combine_multiple_filters() { +// use crate::logical_plan::{and, binary_expr, col, lit, Operator}; +// let filter1 = binary_expr(col("c1"), Operator::Lt, lit(1)); +// let filter2 = binary_expr(col("c2"), Operator::Lt, lit(2)); +// let filter3 = binary_expr(col("c3"), Operator::Lt, lit(3)); +// let result = +// combine_filters(&[filter1.clone(), filter2.clone(), filter3.clone()]); +// assert_eq!(result, Some(and(and(filter1, filter2), filter3))); +// } +// } diff --git a/datafusion/src/datasource/listing.rs b/datafusion/src/datasource/listing.rs new file mode 100644 index 000000000000..1a02136f8503 --- /dev/null +++ b/datafusion/src/datasource/listing.rs @@ -0,0 +1,294 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! A table that uses the files system / table store listing capability +//! to get the list of files to process. + +use std::{any::Any, sync::Arc}; + +use arrow::datatypes::SchemaRef; + +use crate::{ + datasource::format::{self}, + error::Result, + logical_plan::{combine_filters, Expr}, + physical_plan::{common, parquet::ParquetExec, ExecutionPlan, Statistics}, +}; + +use super::{PartitionedFile, TableProvider}; + +/// The supported file types with the associated options. +pub enum FormatOptions { + /// The Apache Parquet file type. + Parquet { + /// Parquet files contain row group statistics in the + /// metadata section. Set true to parse it. This can + /// add a lot of overhead as it requires each file to + /// be opened and partially parsed. + collect_stat: bool, + /// Activate statistics based row group level pruning + enable_pruning: bool, + /// group files to avoid that the number of partitions + /// exceeds this limit + max_partitions: usize, + }, + /// Row oriented text file with newline as row delimiter. + Csv { + /// Set true to indicate that the first line is a header. + has_header: bool, + /// The character seprating values within a row. + 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. + schema_infer_max_rec: Option, + }, + /// New line delimited JSON. + Json { + /// 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. + schema_infer_max_rec: Option, + }, +} + +/// Options for creating a `ListingTable` +pub struct ListingOptions { + /// A suffix on which files should be filtered (leave empty to + /// keep all files on the path) + pub extension: String, + /// The file format + pub format: FormatOptions, +} + +impl ListingOptions { + /// 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 fn infer_schema(&self, _path: &str) -> Result { + match self { + ListingOptions { + format: FormatOptions::Parquet { .. }, + .. + } => { + todo!("list one file and get the schema for it") + } + _ => todo!("other file formats"), + } + } + + fn create_executor( + &self, + schema: SchemaRef, + files: Vec, + projection: &Option>, + batch_size: usize, + filters: &[Expr], + limit: Option, + ) -> Result> { + match self { + ListingOptions { + format: + FormatOptions::Parquet { + collect_stat, + enable_pruning, + max_partitions, + }, + .. + } => { + // 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 *enable_pruning { + combine_filters(filters) + } else { + None + }; + + // collect the statistics if required by the config + let mut files = files; + if *collect_stat { + files = files + .into_iter() + .map(|file| -> Result { + let (_schema, statistics) = + format::parquet::fetch_metadata(&file.path)?; + // TODO use _schema to check that it is valid or for schema merging + Ok(PartitionedFile { + statistics, + path: file.path, + }) + }) + .collect::>>()?; + } + + let (files, statistics) = + format::get_statistics_with_limit(&files, Arc::clone(&schema), limit); + + let partitioned_file_lists = split_files(files, *max_partitions); + + Ok(Arc::new(ParquetExec::try_new_refacto( + partitioned_file_lists, + statistics, + schema, + projection.clone(), + predicate, + limit + .map(|l| std::cmp::min(l, batch_size)) + .unwrap_or(batch_size), + limit, + )?)) + } + _ => todo!(), + } + } +} + +/// 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, + schema: SchemaRef, + options: ListingOptions, +} + +impl ListingTable { + /// Create new table that lists the FS to get the files to scan. + pub fn try_new( + path: impl Into, + // the schema must be resolved before creating the table + schema: SchemaRef, + options: ListingOptions, + ) -> Result { + let path: String = path.into(); + Ok(Self { + path, + schema, + options, + }) + } +} + +impl TableProvider for ListingTable { + fn as_any(&self) -> &dyn Any { + self + } + + fn schema(&self) -> SchemaRef { + Arc::clone(&self.schema) + } + + fn scan( + &self, + projection: &Option>, + batch_size: usize, + filters: &[Expr], + limit: Option, + ) -> Result> { + // 1. list files (with partitions) + let file_list = + pruned_partition_list(&self.path, filters, &self.options.extension)?; + // 2. create the plan + self.options.create_executor( + self.schema(), + file_list, + projection, + batch_size, + filters, + limit, + ) + } +} + +/// Discover the partitions on the given path and prune out files +/// relative to irrelevant partitions using `filters` expressions +fn pruned_partition_list( + // registry: &ObjectStoreRegistry, + path: &str, + _filters: &[Expr], + file_extension: &str, +) -> Result> { + // TODO: parse folder names first to get partitions and apply the `filters` + // to list only relevant ones + Ok(common::build_file_list(path, file_extension)? + .into_iter() + .map(|f| PartitionedFile { + path: f, + statistics: Statistics::default(), + }) + .collect()) +} + +fn split_files( + partitioned_files: Vec, + n: usize, +) -> Vec> { + let mut chunk_size = partitioned_files.len() / n; + if partitioned_files.len() % n > 0 { + chunk_size += 1; + } + partitioned_files + .chunks(chunk_size) + .map(|c| c.to_vec()) + .collect() +} + +#[cfg(test)] +mod tests { + use super::*; + use futures::StreamExt; + + #[tokio::test] + async fn read_small_batches() -> Result<()> { + let table = load_table("alltypes_plain.parquet")?; + let projection = None; + let exec = table.scan(&projection, 2, &[], None)?; + let stream = exec.execute(0).await?; + + let _ = stream + .map(|batch| { + let batch = batch.unwrap(); + assert_eq!(11, batch.num_columns()); + assert_eq!(2, batch.num_rows()); + }) + .fold(0, |acc, _| async move { acc + 1i32 }) + .await; + + // test metadata + assert_eq!(exec.statistics().num_rows, Some(8)); + assert_eq!(exec.statistics().total_byte_size, Some(671)); + + Ok(()) + } + + fn load_table(name: &str) -> Result> { + let testdata = crate::test_util::parquet_test_data(); + let filename = format!("{}/{}", testdata, name); + let opt = ListingOptions { + extension: ".parquet".to_owned(), + format: FormatOptions::Parquet { + collect_stat: false, + enable_pruning: true, + max_partitions: 2, + }, + }; + // here we resolve the schema locally + let schema = opt.infer_schema(&filename)?; + let table = ListingTable::try_new(&filename, schema, opt)?; + Ok(Arc::new(table)) + } +} diff --git a/datafusion/src/datasource/mod.rs b/datafusion/src/datasource/mod.rs index ab70014a4367..c60ae4c3f16e 100644 --- a/datafusion/src/datasource/mod.rs +++ b/datafusion/src/datasource/mod.rs @@ -21,7 +21,9 @@ pub mod avro; pub mod csv; pub mod datasource; pub mod empty; +pub mod format; pub mod json; +pub mod listing; pub mod memory; pub mod object_store; pub mod parquet; diff --git a/datafusion/src/physical_plan/parquet.rs b/datafusion/src/physical_plan/parquet.rs index f4ac4c8fddaf..80eaf53de5e6 100644 --- a/datafusion/src/physical_plan/parquet.rs +++ b/datafusion/src/physical_plan/parquet.rs @@ -189,6 +189,57 @@ impl ParquetExec { )) } + /// Create a new Parquet reader execution plan provided file list and schema + pub fn try_new_refacto( + files: Vec>, + statistics: Statistics, + schema: SchemaRef, + projection: Option>, + predicate: Option, + batch_size: usize, + limit: Option, + ) -> Result { + debug!("Creating ParquetExec, desc: {:?}, projection {:?}, predicate: {:?}, limit: {:?}", + files, projection, predicate, limit); + + let metrics = ExecutionPlanMetricsSet::new(); + + let partitions = files + .into_iter() + .enumerate() + .map(|(i, f)| ParquetPartition::new(f, i, metrics.clone())) + .collect::>(); + + let metrics = ExecutionPlanMetricsSet::new(); + let predicate_creation_errors = + MetricBuilder::new(&metrics).global_counter("num_predicate_creation_errors"); + + let predicate_builder = predicate.and_then(|predicate_expr| { + match PruningPredicate::try_new(&predicate_expr, schema.clone()) { + Ok(predicate_builder) => Some(predicate_builder), + Err(e) => { + debug!( + "Could not create pruning predicate for {:?}: {}", + predicate_expr, e + ); + predicate_creation_errors.add(1); + None + } + } + }); + + Ok(Self::new( + partitions, + schema, + projection, + statistics, + metrics, + predicate_builder, + batch_size, + limit, + )) + } + /// Create a new Parquet reader execution plan with provided partitions and schema #[allow(clippy::too_many_arguments)] pub fn new( From 4ea972e4eb89eeb5cce146f27bf44eca0f03dc99 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 12 Oct 2021 19:24:15 +0200 Subject: [PATCH 02/39] [feat] implement infer_schema to make test pass --- datafusion/src/datasource/format/mod.rs | 1 + datafusion/src/datasource/listing.rs | 30 ++++++++++++++++--------- 2 files changed, 21 insertions(+), 10 deletions(-) diff --git a/datafusion/src/datasource/format/mod.rs b/datafusion/src/datasource/format/mod.rs index 33c0717899de..053f8aa036c7 100644 --- a/datafusion/src/datasource/format/mod.rs +++ b/datafusion/src/datasource/format/mod.rs @@ -30,6 +30,7 @@ use super::PartitionedFile; /// Get all files as well as the summary statistic /// if the optional `limit` is provided, includes only sufficient files /// 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)) pub fn get_statistics_with_limit( all_files: &[PartitionedFile], schema: SchemaRef, diff --git a/datafusion/src/datasource/listing.rs b/datafusion/src/datasource/listing.rs index 1a02136f8503..641bcac04476 100644 --- a/datafusion/src/datasource/listing.rs +++ b/datafusion/src/datasource/listing.rs @@ -24,7 +24,7 @@ use arrow::datatypes::SchemaRef; use crate::{ datasource::format::{self}, - error::Result, + error::{DataFusionError, Result}, logical_plan::{combine_filters, Expr}, physical_plan::{common, parquet::ParquetExec, ExecutionPlan, Statistics}, }; @@ -79,13 +79,23 @@ impl ListingOptions { /// 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 fn infer_schema(&self, _path: &str) -> Result { - match self { - ListingOptions { - format: FormatOptions::Parquet { .. }, - .. - } => { - todo!("list one file and get the schema for it") + pub fn infer_schema(&self, path: &str) -> 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 = common::build_file_list(path, &self.extension)? + .into_iter() + .next() + .ok_or_else(|| { + DataFusionError::Plan(format!( + "No file (with .{} extension) found at path {}", + &self.extension, path + )) + })?; + match self.format { + FormatOptions::Parquet { .. } => { + let (schema, _) = format::parquet::fetch_metadata(&first_file)?; + Ok(Arc::new(schema)) } _ => todo!("other file formats"), } @@ -125,7 +135,7 @@ impl ListingOptions { files = files .into_iter() .map(|file| -> Result { - let (_schema, statistics) = + let (_, statistics) = format::parquet::fetch_metadata(&file.path)?; // TODO use _schema to check that it is valid or for schema merging Ok(PartitionedFile { @@ -281,7 +291,7 @@ mod tests { let opt = ListingOptions { extension: ".parquet".to_owned(), format: FormatOptions::Parquet { - collect_stat: false, + collect_stat: true, enable_pruning: true, max_partitions: 2, }, From ca376076987743e559abfbb157b9aec4eb8e8a18 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 12 Oct 2021 19:24:15 +0200 Subject: [PATCH 03/39] [wip] trying to implement pruned_partition_list --- datafusion/src/datasource/listing.rs | 123 ++++++++++++++++++++++----- 1 file changed, 101 insertions(+), 22 deletions(-) diff --git a/datafusion/src/datasource/listing.rs b/datafusion/src/datasource/listing.rs index 641bcac04476..d8db37ea16db 100644 --- a/datafusion/src/datasource/listing.rs +++ b/datafusion/src/datasource/listing.rs @@ -18,18 +18,22 @@ //! A table that uses the files system / table store listing capability //! to get the list of files to process. -use std::{any::Any, sync::Arc}; +use std::{any::Any, collections::HashSet, sync::Arc}; -use arrow::datatypes::SchemaRef; +use arrow::{ + datatypes::{DataType, Field, Schema, SchemaRef}, + record_batch::RecordBatch, +}; use crate::{ datasource::format::{self}, error::{DataFusionError, Result}, - logical_plan::{combine_filters, Expr}, + logical_plan::{combine_filters, Column, Expr}, + optimizer, physical_plan::{common, parquet::ParquetExec, ExecutionPlan, Statistics}, }; -use super::{PartitionedFile, TableProvider}; +use super::{datasource::TableProviderFilterPushDown, PartitionedFile, TableProvider}; /// The supported file types with the associated options. pub enum FormatOptions { @@ -70,9 +74,16 @@ pub enum FormatOptions { pub struct ListingOptions { /// A suffix on which files should be filtered (leave empty to /// keep all files on the path) - pub extension: String, + pub file_extension: String, /// The file format pub format: FormatOptions, + /// The expected partition column names. + /// For example `Vec["a", "b"]` means that the two first levels of + /// partitioning expected should be named "a" and "b": + /// - If there is a third level of partitioning it will be ignored. + /// - Files that don't follow this partitioning will be ignored. + /// Note that only `DataType::Utf8` is supported for the column type. + pub partitions: Vec, } impl ListingOptions { @@ -83,22 +94,29 @@ impl ListingOptions { // 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 = common::build_file_list(path, &self.extension)? + let first_file = common::build_file_list(path, &self.file_extension)? .into_iter() .next() .ok_or_else(|| { DataFusionError::Plan(format!( "No file (with .{} extension) found at path {}", - &self.extension, path + &self.file_extension, path )) })?; - match self.format { + // Infer the schema according to the rules specific to this file format + let schema = match self.format { FormatOptions::Parquet { .. } => { let (schema, _) = format::parquet::fetch_metadata(&first_file)?; - Ok(Arc::new(schema)) + schema } _ => todo!("other file formats"), + }; + // Add the partition columns to the file schema + let mut fields = schema.fields().clone(); + for part in &self.partitions { + fields.push(Field::new(part, DataType::Utf8, false)); } + Ok(Arc::new(Schema::new(fields))) } fn create_executor( @@ -210,8 +228,12 @@ impl TableProvider for ListingTable { limit: Option, ) -> Result> { // 1. list files (with partitions) - let file_list = - pruned_partition_list(&self.path, filters, &self.options.extension)?; + let file_list = pruned_partition_list( + &self.path, + filters, + &self.options.file_extension, + &self.options.partitions, + )?; // 2. create the plan self.options.create_executor( self.schema(), @@ -222,6 +244,13 @@ impl TableProvider for ListingTable { limit, ) } + + fn supports_filter_pushdown( + &self, + _filter: &Expr, + ) -> Result { + Ok(TableProviderFilterPushDown::Inexact) + } } /// Discover the partitions on the given path and prune out files @@ -229,18 +258,67 @@ impl TableProvider for ListingTable { fn pruned_partition_list( // registry: &ObjectStoreRegistry, path: &str, - _filters: &[Expr], + filters: &[Expr], file_extension: &str, + partition_names: &[String], ) -> Result> { - // TODO: parse folder names first to get partitions and apply the `filters` - // to list only relevant ones - Ok(common::build_file_list(path, file_extension)? - .into_iter() - .map(|f| PartitionedFile { - path: f, - statistics: Statistics::default(), - }) - .collect()) + let list_all = || { + Ok(common::build_file_list(path, file_extension)? + .into_iter() + .map(|f| PartitionedFile { + path: f, + statistics: Statistics::default(), + }) + .collect::>()) + }; + if partition_names.is_empty() { + list_all() + } else { + let mut applicable_exprs = vec![]; + let partition_set = partition_names.iter().collect::>(); + 'expr: for expr in filters { + let mut columns: HashSet = HashSet::new(); + optimizer::utils::expr_to_columns(expr, &mut columns)?; + for col in columns { + if !partition_set.contains(&col.name) { + continue 'expr; + } + } + applicable_exprs.push(expr.clone()); + } + + if applicable_exprs.is_empty() { + list_all() + } else { + // 1) could be to run the filters on the partition values + + // let partition_values = list_partitions(path, partition_names)?; + // let df = ExecutionContext::new() + // .read_table(Arc::new(MemTable::try_new( + // partition_values.schema(), + // vec![vec![partition_values]], + // )?))? + // .filter(combine_filters(&applicable_exprs).unwrap())? + // .collect() + // .await?; + + // this requires `fn scan()` to be async + + // 2) take the filtered partition lines and list the files + // contained in the associated folders + + todo!() + } + } +} + +#[allow(dead_code)] +fn list_partitions( + // registry: &ObjectStoreRegistry, + _path: &str, + _partitions: &[String], +) -> Result { + todo!() } fn split_files( @@ -289,12 +367,13 @@ mod tests { let testdata = crate::test_util::parquet_test_data(); let filename = format!("{}/{}", testdata, name); let opt = ListingOptions { - extension: ".parquet".to_owned(), + file_extension: "parquet".to_owned(), format: FormatOptions::Parquet { collect_stat: true, enable_pruning: true, max_partitions: 2, }, + partitions: vec![], }; // here we resolve the schema locally let schema = opt.infer_schema(&filename)?; From 503427575431d2ad8c5b1480bcf06d6869d42788 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 12 Oct 2021 19:24:15 +0200 Subject: [PATCH 04/39] [typo] --- ballista/rust/scheduler/src/lib.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ballista/rust/scheduler/src/lib.rs b/ballista/rust/scheduler/src/lib.rs index 0a06cd7b4995..e57b967de8a1 100644 --- a/ballista/rust/scheduler/src/lib.rs +++ b/ballista/rust/scheduler/src/lib.rs @@ -269,7 +269,7 @@ impl SchedulerGrpc for SchedulerServer { } } - /// this service would replace get_file_meatadata + /// this service would replace get_file_metadata async fn get_schema( &self, _request: Request, From 8c7321e54079a2bde825f8970a5838d2af972b6e Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 12 Oct 2021 19:24:15 +0200 Subject: [PATCH 05/39] [fix] replace enum with trait for extensibility --- ballista/rust/core/proto/ballista.proto | 6 +- datafusion/src/datasource/format/csv.rs | 66 ++++ datafusion/src/datasource/format/json.rs | 62 ++++ datafusion/src/datasource/format/mod.rs | 40 ++- datafusion/src/datasource/format/parquet.rs | 63 +++- datafusion/src/datasource/listing.rs | 325 ++++++-------------- 6 files changed, 330 insertions(+), 232 deletions(-) create mode 100644 datafusion/src/datasource/format/csv.rs create mode 100644 datafusion/src/datasource/format/json.rs diff --git a/ballista/rust/core/proto/ballista.proto b/ballista/rust/core/proto/ballista.proto index a8d1dd15c639..f92c7d71d1ba 100644 --- a/ballista/rust/core/proto/ballista.proto +++ b/ballista/rust/core/proto/ballista.proto @@ -949,14 +949,14 @@ message GetFileMetadataParams { FileType file_type = 2; } -message ParquetConfig { - // fields of datasource::listing::FormatOptions::Parquet +message ParquetFormat { + // fields of datasource::format::parquet::ParquetFormat } message ListingConfig { string extension = 1; oneof format { - ParquetConfig parquet = 2; + ParquetFormat parquet = 2; // csv, json, ... } } diff --git a/datafusion/src/datasource/format/csv.rs b/datafusion/src/datasource/format/csv.rs new file mode 100644 index 000000000000..2acd6892f84f --- /dev/null +++ b/datafusion/src/datasource/format/csv.rs @@ -0,0 +1,66 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! CSV format abstractions + +use std::sync::Arc; + +use arrow::datatypes::SchemaRef; +use async_trait::async_trait; + +use super::FileFormat; +use crate::datasource::PartitionedFile; +use crate::error::Result; +use crate::logical_plan::Expr; +use crate::physical_plan::ExecutionPlan; +use crate::physical_plan::Statistics; + +/// Character Separated Value `FileFormat` implementation. +pub struct CsvFormat { + /// 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, +} + +#[async_trait] +impl FileFormat for CsvFormat { + async fn infer_schema(&self, _path: &str) -> Result { + todo!() + } + + async fn infer_stats(&self, _path: &str) -> Result { + Ok(Statistics::default()) + } + + async fn create_executor( + &self, + _schema: SchemaRef, + _files: Vec>, + _statistics: Statistics, + _projection: &Option>, + _batch_size: usize, + _filters: &[Expr], + _limit: Option, + ) -> Result> { + todo!() + } +} diff --git a/datafusion/src/datasource/format/json.rs b/datafusion/src/datasource/format/json.rs new file mode 100644 index 000000000000..3f3f9a1c7909 --- /dev/null +++ b/datafusion/src/datasource/format/json.rs @@ -0,0 +1,62 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! Line delimited JSON format abstractions + +use std::sync::Arc; + +use arrow::datatypes::SchemaRef; +use async_trait::async_trait; + +use super::FileFormat; +use crate::datasource::PartitionedFile; +use crate::error::Result; +use crate::logical_plan::Expr; +use crate::physical_plan::ExecutionPlan; +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, +} + +#[async_trait] +impl FileFormat for JsonFormat { + async fn infer_schema(&self, _path: &str) -> Result { + todo!() + } + + async fn infer_stats(&self, _path: &str) -> Result { + Ok(Statistics::default()) + } + + async fn create_executor( + &self, + _schema: SchemaRef, + _files: Vec>, + _statistics: Statistics, + _projection: &Option>, + _batch_size: usize, + _filters: &[Expr], + _limit: Option, + ) -> Result> { + todo!() + } +} diff --git a/datafusion/src/datasource/format/mod.rs b/datafusion/src/datasource/format/mod.rs index 053f8aa036c7..b33fe55c3015 100644 --- a/datafusion/src/datasource/format/mod.rs +++ b/datafusion/src/datasource/format/mod.rs @@ -17,16 +17,50 @@ //! Module containing helper methods for the various file formats -// pub mod csv; -// pub mod json; +pub mod csv; +pub mod json; pub mod parquet; +use std::sync::Arc; + use crate::arrow::datatypes::{Schema, SchemaRef}; +use crate::error::Result; +use crate::logical_plan::Expr; use crate::physical_plan::expressions::{MaxAccumulator, MinAccumulator}; -use crate::physical_plan::{Accumulator, ColumnStatistics, Statistics}; +use crate::physical_plan::{Accumulator, ColumnStatistics, ExecutionPlan, Statistics}; use super::PartitionedFile; +use async_trait::async_trait; + +/// 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 file at the given path and infer its schema + async fn infer_schema(&self, path: &str) -> Result; + + /// Open the file at the given path and infer its statistics + async fn infer_stats(&self, path: &str) -> 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_executor( + &self, + schema: SchemaRef, + files: Vec>, + statistics: Statistics, + projection: &Option>, + batch_size: usize, + filters: &[Expr], + limit: Option, + ) -> Result>; +} + /// Get all files as well as the summary statistic /// if the optional `limit` is provided, includes only sufficient files /// needed to read up to `limit` number of rows diff --git a/datafusion/src/datasource/format/parquet.rs b/datafusion/src/datasource/format/parquet.rs index cb2faae3870d..198d2f13580c 100644 --- a/datafusion/src/datasource/format/parquet.rs +++ b/datafusion/src/datasource/format/parquet.rs @@ -15,21 +15,29 @@ // specific language governing permissions and limitations // under the License. -//! Parquet format helper methods +//! Parquet format abstractions use std::fs::File; use std::sync::Arc; use arrow::datatypes::Schema; +use arrow::datatypes::SchemaRef; +use async_trait::async_trait; use parquet::arrow::ArrowReader; use parquet::arrow::ParquetFileArrowReader; use parquet::file::serialized_reader::SerializedFileReader; use parquet::file::statistics::Statistics as ParquetStatistics; +use super::FileFormat; use super::{create_max_min_accs, get_col_stats}; use crate::arrow::datatypes::{DataType, Field}; +use crate::datasource::PartitionedFile; 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::parquet::ParquetExec; +use crate::physical_plan::ExecutionPlan; use crate::physical_plan::{Accumulator, Statistics}; use crate::scalar::ScalarValue; @@ -156,7 +164,7 @@ fn summarize_min_max( } /// Read and parse the metadata of the Parquet file at location `path` -pub fn fetch_metadata(path: &str) -> Result<(Schema, Statistics)> { +fn fetch_metadata(path: &str) -> Result<(Schema, Statistics)> { let file = File::open(path)?; let file_reader = Arc::new(SerializedFileReader::new(file)?); let mut arrow_reader = ParquetFileArrowReader::new(file_reader); @@ -214,6 +222,57 @@ pub fn fetch_metadata(path: &str) -> Result<(Schema, Statistics)> { Ok((schema, statistics)) } +/// The Apache Parquet `FileFormat` implementation +pub struct ParquetFormat { + /// Activate statistics based row group level pruning + pub enable_pruning: bool, +} + +#[async_trait] +impl FileFormat for ParquetFormat { + async fn infer_schema(&self, path: &str) -> Result { + let (schema, _) = fetch_metadata(path)?; + Ok(Arc::new(schema)) + } + + async fn infer_stats(&self, path: &str) -> Result { + let (_, stats) = fetch_metadata(path)?; + Ok(stats) + } + + async fn create_executor( + &self, + schema: SchemaRef, + files: Vec>, + statistics: Statistics, + projection: &Option>, + batch_size: usize, + filters: &[Expr], + limit: Option, + ) -> 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) + } else { + None + }; + + Ok(Arc::new(ParquetExec::try_new_refacto( + files, + statistics, + schema, + projection.clone(), + predicate, + limit + .map(|l| std::cmp::min(l, batch_size)) + .unwrap_or(batch_size), + limit, + )?)) + } +} + // #[cfg(test)] // mod tests { // use super::*; diff --git a/datafusion/src/datasource/listing.rs b/datafusion/src/datasource/listing.rs index d8db37ea16db..259a44312db6 100644 --- a/datafusion/src/datasource/listing.rs +++ b/datafusion/src/datasource/listing.rs @@ -18,57 +18,21 @@ //! A table that uses the files system / table store listing capability //! to get the list of files to process. -use std::{any::Any, collections::HashSet, sync::Arc}; +use std::{any::Any, sync::Arc}; -use arrow::{ - datatypes::{DataType, Field, Schema, SchemaRef}, - record_batch::RecordBatch, -}; +use arrow::datatypes::SchemaRef; +use futures::{StreamExt, TryStreamExt}; use crate::{ datasource::format::{self}, error::{DataFusionError, Result}, - logical_plan::{combine_filters, Column, Expr}, - optimizer, - physical_plan::{common, parquet::ParquetExec, ExecutionPlan, Statistics}, + logical_plan::Expr, + physical_plan::{common, ExecutionPlan, Statistics}, }; -use super::{datasource::TableProviderFilterPushDown, PartitionedFile, TableProvider}; - -/// The supported file types with the associated options. -pub enum FormatOptions { - /// The Apache Parquet file type. - Parquet { - /// Parquet files contain row group statistics in the - /// metadata section. Set true to parse it. This can - /// add a lot of overhead as it requires each file to - /// be opened and partially parsed. - collect_stat: bool, - /// Activate statistics based row group level pruning - enable_pruning: bool, - /// group files to avoid that the number of partitions - /// exceeds this limit - max_partitions: usize, - }, - /// Row oriented text file with newline as row delimiter. - Csv { - /// Set true to indicate that the first line is a header. - has_header: bool, - /// The character seprating values within a row. - 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. - schema_infer_max_rec: Option, - }, - /// New line delimited JSON. - Json { - /// 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. - schema_infer_max_rec: Option, - }, -} +use super::{ + datasource::TableProviderFilterPushDown, format::FileFormat, PartitionedFile, +}; /// Options for creating a `ListingTable` pub struct ListingOptions { @@ -76,21 +40,29 @@ pub struct ListingOptions { /// keep all files on the path) pub file_extension: String, /// The file format - pub format: FormatOptions, + pub format: Arc, /// The expected partition column names. /// For example `Vec["a", "b"]` means that the two first levels of /// partitioning expected should be named "a" and "b": /// - If there is a third level of partitioning it will be ignored. /// - Files that don't follow this partitioning will be ignored. /// 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. + /// This can add a lot of overhead as it requires files to + /// be opened and partially parsed. + pub collect_stat: bool, + /// Group files to avoid that the number of partitions + /// exceeds this limit + pub max_partitions: usize, } impl ListingOptions { /// 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 fn infer_schema(&self, path: &str) -> Result { + pub async fn infer_schema(&self, path: &str) -> 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 @@ -103,86 +75,7 @@ impl ListingOptions { &self.file_extension, path )) })?; - // Infer the schema according to the rules specific to this file format - let schema = match self.format { - FormatOptions::Parquet { .. } => { - let (schema, _) = format::parquet::fetch_metadata(&first_file)?; - schema - } - _ => todo!("other file formats"), - }; - // Add the partition columns to the file schema - let mut fields = schema.fields().clone(); - for part in &self.partitions { - fields.push(Field::new(part, DataType::Utf8, false)); - } - Ok(Arc::new(Schema::new(fields))) - } - - fn create_executor( - &self, - schema: SchemaRef, - files: Vec, - projection: &Option>, - batch_size: usize, - filters: &[Expr], - limit: Option, - ) -> Result> { - match self { - ListingOptions { - format: - FormatOptions::Parquet { - collect_stat, - enable_pruning, - max_partitions, - }, - .. - } => { - // 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 *enable_pruning { - combine_filters(filters) - } else { - None - }; - - // collect the statistics if required by the config - let mut files = files; - if *collect_stat { - files = files - .into_iter() - .map(|file| -> Result { - let (_, statistics) = - format::parquet::fetch_metadata(&file.path)?; - // TODO use _schema to check that it is valid or for schema merging - Ok(PartitionedFile { - statistics, - path: file.path, - }) - }) - .collect::>>()?; - } - - let (files, statistics) = - format::get_statistics_with_limit(&files, Arc::clone(&schema), limit); - - let partitioned_file_lists = split_files(files, *max_partitions); - - Ok(Arc::new(ParquetExec::try_new_refacto( - partitioned_file_lists, - statistics, - schema, - projection.clone(), - predicate, - limit - .map(|l| std::cmp::min(l, batch_size)) - .unwrap_or(batch_size), - limit, - )?)) - } - _ => todo!(), - } + self.format.infer_schema(&first_file).await } } @@ -211,7 +104,9 @@ impl ListingTable { } } -impl TableProvider for ListingTable { +// TODO add back impl ExecutionPlan +#[allow(dead_code)] +impl ListingTable { fn as_any(&self) -> &dyn Any { self } @@ -220,7 +115,7 @@ impl TableProvider for ListingTable { Arc::clone(&self.schema) } - fn scan( + async fn scan( &self, projection: &Option>, batch_size: usize, @@ -234,15 +129,40 @@ impl TableProvider for ListingTable { &self.options.file_extension, &self.options.partitions, )?; + + // collect the statistics if required by the config + let mut files = file_list; + if self.options.collect_stat { + files = futures::stream::iter(files) + .then(|file| async { + let statistics = self.options.format.infer_stats(&file.path).await?; + Ok(PartitionedFile { + statistics, + path: file.path, + }) as Result + }) + .try_collect::>() + .await?; + } + + let (files, statistics) = + format::get_statistics_with_limit(&files, self.schema(), limit); + + let partitioned_file_lists = split_files(files, self.options.max_partitions); + // 2. create the plan - self.options.create_executor( - self.schema(), - file_list, - projection, - batch_size, - filters, - limit, - ) + self.options + .format + .create_executor( + self.schema(), + partitioned_file_lists, + statistics, + projection, + batch_size, + filters, + limit, + ) + .await } fn supports_filter_pushdown( @@ -258,7 +178,7 @@ impl TableProvider for ListingTable { fn pruned_partition_list( // registry: &ObjectStoreRegistry, path: &str, - filters: &[Expr], + _filters: &[Expr], file_extension: &str, partition_names: &[String], ) -> Result> { @@ -274,53 +194,10 @@ fn pruned_partition_list( if partition_names.is_empty() { list_all() } else { - let mut applicable_exprs = vec![]; - let partition_set = partition_names.iter().collect::>(); - 'expr: for expr in filters { - let mut columns: HashSet = HashSet::new(); - optimizer::utils::expr_to_columns(expr, &mut columns)?; - for col in columns { - if !partition_set.contains(&col.name) { - continue 'expr; - } - } - applicable_exprs.push(expr.clone()); - } - - if applicable_exprs.is_empty() { - list_all() - } else { - // 1) could be to run the filters on the partition values - - // let partition_values = list_partitions(path, partition_names)?; - // let df = ExecutionContext::new() - // .read_table(Arc::new(MemTable::try_new( - // partition_values.schema(), - // vec![vec![partition_values]], - // )?))? - // .filter(combine_filters(&applicable_exprs).unwrap())? - // .collect() - // .await?; - - // this requires `fn scan()` to be async - - // 2) take the filtered partition lines and list the files - // contained in the associated folders - - todo!() - } + todo!("use filters to prune partitions") } } -#[allow(dead_code)] -fn list_partitions( - // registry: &ObjectStoreRegistry, - _path: &str, - _partitions: &[String], -) -> Result { - todo!() -} - fn split_files( partitioned_files: Vec, n: usize, @@ -337,47 +214,47 @@ fn split_files( #[cfg(test)] mod tests { - use super::*; - use futures::StreamExt; - - #[tokio::test] - async fn read_small_batches() -> Result<()> { - let table = load_table("alltypes_plain.parquet")?; - let projection = None; - let exec = table.scan(&projection, 2, &[], None)?; - let stream = exec.execute(0).await?; - - let _ = stream - .map(|batch| { - let batch = batch.unwrap(); - assert_eq!(11, batch.num_columns()); - assert_eq!(2, batch.num_rows()); - }) - .fold(0, |acc, _| async move { acc + 1i32 }) - .await; - - // test metadata - assert_eq!(exec.statistics().num_rows, Some(8)); - assert_eq!(exec.statistics().total_byte_size, Some(671)); - - Ok(()) - } - - fn load_table(name: &str) -> Result> { - let testdata = crate::test_util::parquet_test_data(); - let filename = format!("{}/{}", testdata, name); - let opt = ListingOptions { - file_extension: "parquet".to_owned(), - format: FormatOptions::Parquet { - collect_stat: true, - enable_pruning: true, - max_partitions: 2, - }, - partitions: vec![], - }; - // here we resolve the schema locally - let schema = opt.infer_schema(&filename)?; - let table = ListingTable::try_new(&filename, schema, opt)?; - Ok(Arc::new(table)) - } + // use super::*; + // use futures::StreamExt; + + // #[tokio::test] + // async fn read_small_batches() -> Result<()> { + // let table = load_table("alltypes_plain.parquet").await?; + // let projection = None; + // let exec = table.scan(&projection, 2, &[], None)?; + // let stream = exec.execute(0).await?; + + // let _ = stream + // .map(|batch| { + // let batch = batch.unwrap(); + // assert_eq!(11, batch.num_columns()); + // assert_eq!(2, batch.num_rows()); + // }) + // .fold(0, |acc, _| async move { acc + 1i32 }) + // .await; + + // // test metadata + // assert_eq!(exec.statistics().num_rows, Some(8)); + // assert_eq!(exec.statistics().total_byte_size, Some(671)); + + // Ok(()) + // } + + // async fn load_table(name: &str) -> Result> { + // let testdata = crate::test_util::parquet_test_data(); + // let filename = format!("{}/{}", testdata, name); + // let opt = ListingOptions { + // file_extension: "parquet".to_owned(), + // format: Arc::new(format::parquet::ParquetFormat { + // enable_pruning: true, + // }), + // partitions: vec![], + // max_partitions: 2, + // collect_stat: true, + // }; + // // here we resolve the schema locally + // let schema = opt.infer_schema(&filename).await?; + // let table = ListingTable::try_new(&filename, schema, opt)?; + // Ok(Arc::new(table)) + // } } From c69faadbde0a06453f4c4491a50900447c338bdf Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 12 Oct 2021 19:24:15 +0200 Subject: [PATCH 06/39] [fix] add partition cols to infered schema --- datafusion/src/datasource/listing.rs | 95 +++++++++++++++------------- 1 file changed, 51 insertions(+), 44 deletions(-) diff --git a/datafusion/src/datasource/listing.rs b/datafusion/src/datasource/listing.rs index 259a44312db6..3527a70074f3 100644 --- a/datafusion/src/datasource/listing.rs +++ b/datafusion/src/datasource/listing.rs @@ -20,7 +20,8 @@ use std::{any::Any, sync::Arc}; -use arrow::datatypes::SchemaRef; +use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; +use async_trait::async_trait; use futures::{StreamExt, TryStreamExt}; use crate::{ @@ -32,6 +33,7 @@ use crate::{ use super::{ datasource::TableProviderFilterPushDown, format::FileFormat, PartitionedFile, + TableProvider, }; /// Options for creating a `ListingTable` @@ -75,7 +77,13 @@ impl ListingOptions { &self.file_extension, path )) })?; - self.format.infer_schema(&first_file).await + let file_schema = self.format.infer_schema(&first_file).await?; + // Add the partition columns to the file schema + let mut fields = file_schema.fields().clone(); + for part in &self.partitions { + fields.push(Field::new(part, DataType::Utf8, false)); + } + Ok(Arc::new(Schema::new(fields))) } } @@ -104,9 +112,8 @@ impl ListingTable { } } -// TODO add back impl ExecutionPlan -#[allow(dead_code)] -impl ListingTable { +#[async_trait] +impl TableProvider for ListingTable { fn as_any(&self) -> &dyn Any { self } @@ -122,7 +129,7 @@ impl ListingTable { filters: &[Expr], limit: Option, ) -> Result> { - // 1. list files (with partitions) + // list files (with partitions) let file_list = pruned_partition_list( &self.path, filters, @@ -214,47 +221,47 @@ fn split_files( #[cfg(test)] mod tests { - // use super::*; - // use futures::StreamExt; + use super::*; + use futures::StreamExt; - // #[tokio::test] - // async fn read_small_batches() -> Result<()> { - // let table = load_table("alltypes_plain.parquet").await?; - // let projection = None; - // let exec = table.scan(&projection, 2, &[], None)?; - // let stream = exec.execute(0).await?; + #[tokio::test] + async fn read_small_batches() -> Result<()> { + let table = load_table("alltypes_plain.parquet").await?; + let projection = None; + let exec = table.scan(&projection, 2, &[], None).await?; + let stream = exec.execute(0).await?; - // let _ = stream - // .map(|batch| { - // let batch = batch.unwrap(); - // assert_eq!(11, batch.num_columns()); - // assert_eq!(2, batch.num_rows()); - // }) - // .fold(0, |acc, _| async move { acc + 1i32 }) - // .await; + let _ = stream + .map(|batch| { + let batch = batch.unwrap(); + assert_eq!(11, batch.num_columns()); + assert_eq!(2, batch.num_rows()); + }) + .fold(0, |acc, _| async move { acc + 1i32 }) + .await; - // // test metadata - // assert_eq!(exec.statistics().num_rows, Some(8)); - // assert_eq!(exec.statistics().total_byte_size, Some(671)); + // test metadata + assert_eq!(exec.statistics().num_rows, Some(8)); + assert_eq!(exec.statistics().total_byte_size, Some(671)); - // Ok(()) - // } + Ok(()) + } - // async fn load_table(name: &str) -> Result> { - // let testdata = crate::test_util::parquet_test_data(); - // let filename = format!("{}/{}", testdata, name); - // let opt = ListingOptions { - // file_extension: "parquet".to_owned(), - // format: Arc::new(format::parquet::ParquetFormat { - // enable_pruning: true, - // }), - // partitions: vec![], - // max_partitions: 2, - // collect_stat: true, - // }; - // // here we resolve the schema locally - // let schema = opt.infer_schema(&filename).await?; - // let table = ListingTable::try_new(&filename, schema, opt)?; - // Ok(Arc::new(table)) - // } + async fn load_table(name: &str) -> Result> { + let testdata = crate::test_util::parquet_test_data(); + let filename = format!("{}/{}", testdata, name); + let opt = ListingOptions { + file_extension: "parquet".to_owned(), + format: Arc::new(format::parquet::ParquetFormat { + enable_pruning: true, + }), + partitions: vec![], + max_partitions: 2, + collect_stat: true, + }; + // here we resolve the schema locally + let schema = opt.infer_schema(&filename).await?; + let table = ListingTable::try_new(&filename, schema, opt)?; + Ok(Arc::new(table)) + } } From 551ceb4ca277f16af00eebf27a5e8e5af30004aa Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 12 Oct 2021 19:24:15 +0200 Subject: [PATCH 07/39] [feat] forked file format executors avro still missing --- datafusion/src/datasource/format/csv.rs | 59 +- datafusion/src/datasource/format/json.rs | 55 +- datafusion/src/datasource/format/mod.rs | 10 +- datafusion/src/datasource/format/parquet.rs | 19 +- datafusion/src/datasource/listing.rs | 54 +- datafusion/src/physical_plan/format/csv.rs | 302 +++++++ datafusion/src/physical_plan/format/json.rs | 324 +++++++ datafusion/src/physical_plan/format/mod.rs | 26 + .../src/physical_plan/format/parquet.rs | 836 ++++++++++++++++++ datafusion/src/physical_plan/mod.rs | 1 + datafusion/src/physical_plan/parquet.rs | 51 -- 11 files changed, 1642 insertions(+), 95 deletions(-) create mode 100644 datafusion/src/physical_plan/format/csv.rs create mode 100644 datafusion/src/physical_plan/format/json.rs create mode 100644 datafusion/src/physical_plan/format/mod.rs create mode 100644 datafusion/src/physical_plan/format/parquet.rs diff --git a/datafusion/src/datasource/format/csv.rs b/datafusion/src/datasource/format/csv.rs index 2acd6892f84f..0c1db35f1d18 100644 --- a/datafusion/src/datasource/format/csv.rs +++ b/datafusion/src/datasource/format/csv.rs @@ -19,13 +19,17 @@ use std::sync::Arc; -use arrow::datatypes::SchemaRef; +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::{FileFormat, StringStream}; use crate::datasource::PartitionedFile; use crate::error::Result; use crate::logical_plan::Expr; +use crate::physical_plan::format::CsvExec; use crate::physical_plan::ExecutionPlan; use crate::physical_plan::Statistics; @@ -38,13 +42,34 @@ pub struct CsvFormat { /// 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, + pub schema_infer_max_rec: Option, } #[async_trait] impl FileFormat for CsvFormat { - async fn infer_schema(&self, _path: &str) -> Result { - todo!() + async fn infer_schema(&self, mut paths: StringStream) -> 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)?, + self.delimiter, + Some(records_to_read), + self.has_header, + )?; + if records_read == 0 { + continue; + } + schemas.push(schema.clone()); + records_to_read -= records_read; + if records_to_read == 0 { + break; + } + } + + let merged_schema = Schema::try_merge(schemas)?; + Ok(Arc::new(merged_schema)) } async fn infer_stats(&self, _path: &str) -> Result { @@ -53,14 +78,24 @@ impl FileFormat for CsvFormat { async fn create_executor( &self, - _schema: SchemaRef, - _files: Vec>, - _statistics: Statistics, - _projection: &Option>, - _batch_size: usize, + schema: SchemaRef, + files: Vec>, + statistics: Statistics, + projection: &Option>, + batch_size: usize, _filters: &[Expr], - _limit: Option, + limit: Option, ) -> Result> { - todo!() + let exec = CsvExec::try_new( + files.into_iter().flatten().map(|f| f.path).collect(), + statistics, + schema, + self.has_header, + self.delimiter, + projection.clone(), + batch_size, + limit, + )?; + Ok(Arc::new(exec)) } } diff --git a/datafusion/src/datasource/format/json.rs b/datafusion/src/datasource/format/json.rs index 3f3f9a1c7909..521de52cbf71 100644 --- a/datafusion/src/datasource/format/json.rs +++ b/datafusion/src/datasource/format/json.rs @@ -17,15 +17,22 @@ //! Line delimited JSON format abstractions +use std::io::BufReader; use std::sync::Arc; +use arrow::datatypes::Schema; use arrow::datatypes::SchemaRef; +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::{FileFormat, StringStream}; use crate::datasource::PartitionedFile; use crate::error::Result; use crate::logical_plan::Expr; +use crate::physical_plan::format::NdJsonExec; use crate::physical_plan::ExecutionPlan; use crate::physical_plan::Statistics; @@ -34,13 +41,31 @@ 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, + pub schema_infer_max_rec: Option, } #[async_trait] impl FileFormat for JsonFormat { - async fn infer_schema(&self, _path: &str) -> Result { - todo!() + async fn infer_schema(&self, mut paths: StringStream) -> 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); + let iter = ValueIter::new(&mut reader, None); + let schema = infer_json_schema_from_iterator(iter.take_while(|_| { + let should_take = records_to_read > 0; + records_to_read -= 1; + should_take + }))?; + if records_to_read == 0 { + break; + } + schemas.push(schema); + } + + let schema = Schema::try_merge(schemas)?; + Ok(Arc::new(schema)) } async fn infer_stats(&self, _path: &str) -> Result { @@ -49,14 +74,22 @@ impl FileFormat for JsonFormat { async fn create_executor( &self, - _schema: SchemaRef, - _files: Vec>, - _statistics: Statistics, - _projection: &Option>, - _batch_size: usize, + schema: SchemaRef, + files: Vec>, + statistics: Statistics, + projection: &Option>, + batch_size: usize, _filters: &[Expr], - _limit: Option, + limit: Option, ) -> Result> { - todo!() + let exec = NdJsonExec::try_new( + files.into_iter().flatten().map(|f| f.path).collect(), + statistics, + schema, + projection.clone(), + batch_size, + limit, + )?; + Ok(Arc::new(exec)) } } diff --git a/datafusion/src/datasource/format/mod.rs b/datafusion/src/datasource/format/mod.rs index b33fe55c3015..e4e2c7550575 100644 --- a/datafusion/src/datasource/format/mod.rs +++ b/datafusion/src/datasource/format/mod.rs @@ -21,6 +21,7 @@ pub mod csv; pub mod json; pub mod parquet; +use std::pin::Pin; use std::sync::Arc; use crate::arrow::datatypes::{Schema, SchemaRef}; @@ -32,14 +33,19 @@ use crate::physical_plan::{Accumulator, ColumnStatistics, ExecutionPlan, Statist use super::PartitionedFile; 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>>; /// 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 file at the given path and infer its schema - async fn infer_schema(&self, path: &str) -> Result; + /// Open the files at the paths provided by iterator and infer the + /// common schema + async fn infer_schema(&self, paths: StringStream) -> Result; /// Open the file at the given path and infer its statistics async fn infer_stats(&self, path: &str) -> Result; diff --git a/datafusion/src/datasource/format/parquet.rs b/datafusion/src/datasource/format/parquet.rs index 198d2f13580c..34f1f1ebdb66 100644 --- a/datafusion/src/datasource/format/parquet.rs +++ b/datafusion/src/datasource/format/parquet.rs @@ -23,20 +23,22 @@ use std::sync::Arc; use arrow::datatypes::Schema; use arrow::datatypes::SchemaRef; use async_trait::async_trait; +use futures::stream::StreamExt; use parquet::arrow::ArrowReader; use parquet::arrow::ParquetFileArrowReader; use parquet::file::serialized_reader::SerializedFileReader; use parquet::file::statistics::Statistics as ParquetStatistics; use super::FileFormat; -use super::{create_max_min_accs, get_col_stats}; +use super::{create_max_min_accs, get_col_stats, StringStream}; use crate::arrow::datatypes::{DataType, Field}; use crate::datasource::PartitionedFile; +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::parquet::ParquetExec; +use crate::physical_plan::format::ParquetExec; use crate::physical_plan::ExecutionPlan; use crate::physical_plan::{Accumulator, Statistics}; use crate::scalar::ScalarValue; @@ -230,8 +232,15 @@ pub struct ParquetFormat { #[async_trait] impl FileFormat for ParquetFormat { - async fn infer_schema(&self, path: &str) -> Result { - let (schema, _) = fetch_metadata(path)?; + async fn infer_schema(&self, mut paths: StringStream) -> 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(Arc::new(schema)) } @@ -259,7 +268,7 @@ impl FileFormat for ParquetFormat { None }; - Ok(Arc::new(ParquetExec::try_new_refacto( + Ok(Arc::new(ParquetExec::try_new( files, statistics, schema, diff --git a/datafusion/src/datasource/listing.rs b/datafusion/src/datasource/listing.rs index 3527a70074f3..a33f47b64a17 100644 --- a/datafusion/src/datasource/listing.rs +++ b/datafusion/src/datasource/listing.rs @@ -26,7 +26,7 @@ use futures::{StreamExt, TryStreamExt}; use crate::{ datasource::format::{self}, - error::{DataFusionError, Result}, + error::Result, logical_plan::Expr, physical_plan::{common, ExecutionPlan, Statistics}, }; @@ -65,19 +65,9 @@ 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 { - // 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 = common::build_file_list(path, &self.file_extension)? - .into_iter() - .next() - .ok_or_else(|| { - DataFusionError::Plan(format!( - "No file (with .{} extension) found at path {}", - &self.file_extension, path - )) - })?; - let file_schema = self.format.infer_schema(&first_file).await?; + let files = + futures::stream::iter(common::build_file_list(path, &self.file_extension)?); + let file_schema = self.format.infer_schema(Box::pin(files)).await?; // Add the partition columns to the file schema let mut fields = file_schema.fields().clone(); for part in &self.partitions { @@ -224,6 +214,42 @@ mod tests { use super::*; use futures::StreamExt; + #[test] + fn test_split_files() { + let files = vec![ + PartitionedFile::from("a".to_string()), + PartitionedFile::from("b".to_string()), + PartitionedFile::from("c".to_string()), + PartitionedFile::from("d".to_string()), + PartitionedFile::from("e".to_string()), + ]; + + let chunks = split_files(files.clone(), 1); + assert_eq!(1, chunks.len()); + assert_eq!(5, chunks[0].len()); + + let chunks = split_files(files.clone(), 2); + assert_eq!(2, chunks.len()); + assert_eq!(3, chunks[0].len()); + assert_eq!(2, chunks[1].len()); + + let chunks = split_files(files.clone(), 5); + assert_eq!(5, chunks.len()); + assert_eq!(1, chunks[0].len()); + assert_eq!(1, chunks[1].len()); + assert_eq!(1, chunks[2].len()); + assert_eq!(1, chunks[3].len()); + assert_eq!(1, chunks[4].len()); + + let chunks = split_files(files, 123); + assert_eq!(5, chunks.len()); + assert_eq!(1, chunks[0].len()); + assert_eq!(1, chunks[1].len()); + assert_eq!(1, chunks[2].len()); + assert_eq!(1, chunks[3].len()); + assert_eq!(1, chunks[4].len()); + } + #[tokio::test] async fn read_small_batches() -> Result<()> { let table = load_table("alltypes_plain.parquet").await?; diff --git a/datafusion/src/physical_plan/format/csv.rs b/datafusion/src/physical_plan/format/csv.rs new file mode 100644 index 000000000000..0ed26f463152 --- /dev/null +++ b/datafusion/src/physical_plan/format/csv.rs @@ -0,0 +1,302 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! Execution plan for reading CSV files + +use crate::error::{DataFusionError, Result}; +use crate::physical_plan::{ + DisplayFormatType, ExecutionPlan, Partitioning, RecordBatchStream, + SendableRecordBatchStream, Statistics, +}; +use arrow::csv; +use arrow::datatypes::{Schema, SchemaRef}; +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; +use std::task::{Context, Poll}; + +use async_trait::async_trait; + +/// Execution plan for scanning a CSV file +#[derive(Debug, Clone)] +pub struct CsvExec { + /// List of data files + files: Vec, + /// Schema representing the CSV file + schema: SchemaRef, + /// Provided statistics + statistics: Statistics, + /// Does the CSV file have a header? + has_header: bool, + /// An optional column delimiter. Defaults to `b','` + delimiter: Option, + /// Optional projection for which columns to load + projection: Option>, + /// Schema after the projection has been applied + projected_schema: SchemaRef, + /// Batch size + batch_size: usize, + /// Limit in nr. of rows + limit: Option, +} + +impl CsvExec { + /// Create a new CSV reader execution plan provided file list and schema + /// TODO: support partitiondd file list (Vec>) + #[allow(clippy::too_many_arguments)] + pub fn try_new( + files: Vec, + statistics: Statistics, + schema: SchemaRef, + has_header: bool, + delimiter: u8, + projection: Option>, + batch_size: usize, + limit: Option, + ) -> Result { + let projected_schema = match &projection { + None => Arc::clone(&schema), + Some(p) => Arc::new(Schema::new( + p.iter().map(|i| schema.field(*i).clone()).collect(), + )), + }; + + Ok(Self { + files, + schema, + statistics, + has_header, + delimiter: Some(delimiter), + projection, + projected_schema, + batch_size, + limit, + }) + } +} + +#[async_trait] +impl ExecutionPlan for CsvExec { + /// Return a reference to Any that can be used for downcasting + fn as_any(&self) -> &dyn Any { + self + } + + /// Get the schema for this execution plan + fn schema(&self) -> SchemaRef { + self.projected_schema.clone() + } + + /// Get the output partitioning of this plan + fn output_partitioning(&self) -> Partitioning { + Partitioning::UnknownPartitioning(self.files.len()) + } + + fn children(&self) -> Vec> { + // this is a leaf node and has no children + vec![] + } + + fn with_new_children( + &self, + children: Vec>, + ) -> Result> { + if children.is_empty() { + Ok(Arc::new(self.clone())) + } else { + Err(DataFusionError::Internal(format!( + "Children cannot be replaced in {:?}", + self + ))) + } + } + + async fn execute(&self, partition: usize) -> Result { + Ok(Box::pin(CsvStream::try_new( + &self.files[partition], + self.schema.clone(), + self.has_header, + self.delimiter, + &self.projection, + self.batch_size, + self.limit, + )?)) + } + + fn fmt_as( + &self, + t: DisplayFormatType, + f: &mut std::fmt::Formatter, + ) -> std::fmt::Result { + match t { + DisplayFormatType::Default => { + write!( + f, + "CsvExec: has_header={}, batch_size={}, limit={:?}, partitions=[{}]", + self.has_header, + self.batch_size, + self.limit, + self.files.join(", ") + ) + } + } + } + + fn statistics(&self) -> Statistics { + self.statistics.clone() + } +} + +/// Iterator over batches +struct CsvStream { + /// Arrow CSV reader + reader: csv::Reader, +} +impl CsvStream { + /// Create an iterator for a CSV file + pub fn try_new( + filename: &str, + schema: SchemaRef, + has_header: bool, + delimiter: Option, + projection: &Option>, + batch_size: usize, + limit: Option, + ) -> Result { + let file = File::open(filename)?; + 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( + reader: R, + schema: SchemaRef, + has_header: bool, + delimiter: Option, + projection: &Option>, + batch_size: usize, + limit: Option, + ) -> Result> { + let start_line = if has_header { 1 } else { 0 }; + let bounds = limit.map(|x| (0, x + start_line)); + + let reader = csv::Reader::new( + reader, + schema, + has_header, + delimiter, + batch_size, + bounds, + projection.clone(), + ); + + Ok(Self { reader }) + } +} + +impl Stream for CsvStream { + type Item = ArrowResult; + + fn poll_next( + mut self: Pin<&mut Self>, + _: &mut Context<'_>, + ) -> Poll> { + Poll::Ready(self.reader.next()) + } +} + +impl RecordBatchStream for CsvStream { + /// Get the schema + fn schema(&self) -> SchemaRef { + self.reader.schema() + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::test::aggr_test_schema; + use futures::StreamExt; + + #[tokio::test] + async fn csv_exec_with_projection() -> Result<()> { + let schema = aggr_test_schema(); + let testdata = crate::test_util::arrow_test_data(); + let filename = "aggregate_test_100.csv"; + let path = format!("{}/csv/{}", testdata, filename); + let csv = CsvExec::try_new( + vec![path], + Statistics::default(), + schema, + true, + b',', + Some(vec![0, 2, 4]), + 1024, + None, + )?; + assert_eq!(13, csv.schema.fields().len()); + assert_eq!(3, csv.projected_schema.fields().len()); + assert_eq!(3, csv.schema().fields().len()); + let mut stream = csv.execute(0).await?; + let batch = stream.next().await.unwrap()?; + assert_eq!(3, batch.num_columns()); + let batch_schema = batch.schema(); + assert_eq!(3, batch_schema.fields().len()); + assert_eq!("c1", batch_schema.field(0).name()); + assert_eq!("c3", batch_schema.field(1).name()); + assert_eq!("c5", batch_schema.field(2).name()); + Ok(()) + } + + #[tokio::test] + async fn csv_exec_without_projection() -> Result<()> { + let schema = aggr_test_schema(); + let testdata = crate::test_util::arrow_test_data(); + let filename = "aggregate_test_100.csv"; + let path = format!("{}/csv/{}", testdata, filename); + let csv = CsvExec::try_new( + vec![path], + Statistics::default(), + schema, + true, + b',', + None, + 1024, + None, + )?; + assert_eq!(13, csv.schema.fields().len()); + assert_eq!(13, csv.projected_schema.fields().len()); + assert_eq!(13, csv.schema().fields().len()); + let mut it = csv.execute(0).await?; + let batch = it.next().await.unwrap()?; + assert_eq!(13, batch.num_columns()); + let batch_schema = batch.schema(); + assert_eq!(13, batch_schema.fields().len()); + assert_eq!("c1", batch_schema.field(0).name()); + assert_eq!("c2", batch_schema.field(1).name()); + assert_eq!("c3", batch_schema.field(2).name()); + Ok(()) + } +} diff --git a/datafusion/src/physical_plan/format/json.rs b/datafusion/src/physical_plan/format/json.rs new file mode 100644 index 000000000000..0dbcc522f937 --- /dev/null +++ b/datafusion/src/physical_plan/format/json.rs @@ -0,0 +1,324 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! Execution plan for reading line-delimited JSON files +use async_trait::async_trait; +use futures::Stream; + +use crate::error::{DataFusionError, Result}; +use crate::physical_plan::{ + DisplayFormatType, ExecutionPlan, Partitioning, RecordBatchStream, + SendableRecordBatchStream, Statistics, +}; +use arrow::{ + datatypes::{Schema, SchemaRef}, + error::Result as ArrowResult, + json, + record_batch::RecordBatch, +}; +use std::any::Any; +use std::fs::File; +use std::{ + io::Read, + pin::Pin, + sync::Arc, + task::{Context, Poll}, +}; + +/// Execution plan for scanning NdJson data source +#[derive(Debug, Clone)] +pub struct NdJsonExec { + files: Vec, + statistics: Statistics, + schema: SchemaRef, + projection: Option>, + projected_schema: SchemaRef, + batch_size: usize, + limit: Option, +} + +impl NdJsonExec { + /// Create a new JSON reader execution plan provided file list and schema + /// TODO: support partitiondd file list (Vec>) + pub fn try_new( + files: Vec, + statistics: Statistics, + schema: SchemaRef, + projection: Option>, + batch_size: usize, + limit: Option, + ) -> Result { + let projected_schema = match &projection { + None => Arc::clone(&schema), + Some(p) => Arc::new(Schema::new( + p.iter().map(|i| schema.field(*i).clone()).collect(), + )), + }; + + Ok(Self { + files, + statistics, + schema, + projection, + projected_schema, + batch_size, + limit, + }) + } +} + +#[async_trait] +impl ExecutionPlan for NdJsonExec { + fn as_any(&self) -> &dyn Any { + self + } + + fn schema(&self) -> SchemaRef { + self.projected_schema.clone() + } + + fn output_partitioning(&self) -> Partitioning { + Partitioning::UnknownPartitioning(self.files.len()) + } + + fn children(&self) -> Vec> { + Vec::new() + } + + fn with_new_children( + &self, + children: Vec>, + ) -> Result> { + if children.is_empty() { + Ok(Arc::new(self.clone()) as Arc) + } else { + Err(DataFusionError::Internal(format!( + "Children cannot be replaced in {:?}", + self + ))) + } + } + + 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, + ))) + } + + fn fmt_as( + &self, + t: DisplayFormatType, + f: &mut std::fmt::Formatter, + ) -> std::fmt::Result { + match t { + DisplayFormatType::Default => { + write!( + f, + "JsonExec: batch_size={}, limit={:?}, partitions=[{}]", + self.batch_size, + self.limit, + self.files.join(", ") + ) + } + } + } + + fn statistics(&self) -> Statistics { + self.statistics.clone() + } +} + +struct NdJsonStream { + reader: json::Reader, + remain: Option, +} + +impl NdJsonStream { + fn new(reader: json::Reader, limit: Option) -> Self { + Self { + reader, + remain: limit, + } + } +} + +impl Stream for NdJsonStream { + type Item = ArrowResult; + + fn poll_next( + mut self: Pin<&mut Self>, + _cx: &mut Context<'_>, + ) -> Poll> { + if let Some(remain) = self.remain.as_mut() { + if *remain < 1 { + return Poll::Ready(None); + } + } + + Poll::Ready(match self.reader.next() { + Ok(Some(item)) => { + if let Some(remain) = self.remain.as_mut() { + if *remain >= item.num_rows() { + *remain -= item.num_rows(); + Some(Ok(item)) + } else { + let len = *remain; + *remain = 0; + Some(Ok(RecordBatch::try_new( + item.schema(), + item.columns() + .iter() + .map(|column| column.slice(0, len)) + .collect(), + )?)) + } + } else { + Some(Ok(item)) + } + } + Ok(None) => None, + Err(err) => Some(Err(err)), + }) + } +} + +impl RecordBatchStream for NdJsonStream { + fn schema(&self) -> SchemaRef { + self.reader.schema() + } +} + +#[cfg(test)] +mod tests { + use futures::StreamExt; + + use crate::datasource::format::{json::JsonFormat, FileFormat}; + + 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 + } + + #[tokio::test] + async fn nd_json_exec_file_without_projection() -> Result<()> { + use arrow::datatypes::DataType; + let path = format!("{}/1.json", TEST_DATA_BASE); + let exec = NdJsonExec::try_new( + vec![path.clone()], + Default::default(), + infer_schema(path).await?, + None, + 1024, + Some(3), + )?; + + // TODO: this is not where schema inference should be tested + + let inferred_schema = exec.schema(); + assert_eq!(inferred_schema.fields().len(), 4); + + // a,b,c,d should be inferred + inferred_schema.field_with_name("a").unwrap(); + inferred_schema.field_with_name("b").unwrap(); + inferred_schema.field_with_name("c").unwrap(); + inferred_schema.field_with_name("d").unwrap(); + + assert_eq!( + inferred_schema.field_with_name("a").unwrap().data_type(), + &DataType::Int64 + ); + assert!(matches!( + inferred_schema.field_with_name("b").unwrap().data_type(), + DataType::List(_) + )); + assert_eq!( + inferred_schema.field_with_name("d").unwrap().data_type(), + &DataType::Utf8 + ); + + let mut it = exec.execute(0).await?; + let batch = it.next().await.unwrap()?; + + assert_eq!(batch.num_rows(), 3); + let values = batch + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(values.value(0), 1); + assert_eq!(values.value(1), -10); + assert_eq!(values.value(2), 2); + + Ok(()) + } + + #[tokio::test] + async fn nd_json_exec_file_projection() -> Result<()> { + let path = format!("{}/1.json", TEST_DATA_BASE); + let exec = NdJsonExec::try_new( + vec![path.clone()], + Default::default(), + infer_schema(path).await?, + Some(vec![0, 2]), + 1024, + None, + )?; + let inferred_schema = exec.schema(); + assert_eq!(inferred_schema.fields().len(), 2); + + inferred_schema.field_with_name("a").unwrap(); + inferred_schema.field_with_name("b").unwrap_err(); + inferred_schema.field_with_name("c").unwrap(); + inferred_schema.field_with_name("d").unwrap_err(); + + let mut it = exec.execute(0).await?; + let batch = it.next().await.unwrap()?; + + assert_eq!(batch.num_rows(), 4); + let values = batch + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(values.value(0), 1); + assert_eq!(values.value(1), -10); + assert_eq!(values.value(2), 2); + Ok(()) + } +} diff --git a/datafusion/src/physical_plan/format/mod.rs b/datafusion/src/physical_plan/format/mod.rs new file mode 100644 index 000000000000..cd6cebe60338 --- /dev/null +++ b/datafusion/src/physical_plan/format/mod.rs @@ -0,0 +1,26 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! Execution plans that read file formats + +mod csv; +mod json; +mod parquet; + +pub use self::parquet::ParquetExec; +pub use csv::CsvExec; +pub use json::NdJsonExec; diff --git a/datafusion/src/physical_plan/format/parquet.rs b/datafusion/src/physical_plan/format/parquet.rs new file mode 100644 index 000000000000..fb622b5217ed --- /dev/null +++ b/datafusion/src/physical_plan/format/parquet.rs @@ -0,0 +1,836 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! 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::{ + error::{DataFusionError, Result}, + logical_plan::{Column, Expr}, + physical_optimizer::pruning::{PruningPredicate, PruningStatistics}, + physical_plan::{ + metrics::{self, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}, + stream::RecordBatchReceiverStream, + DisplayFormatType, ExecutionPlan, Partitioning, SendableRecordBatchStream, + Statistics, + }, + scalar::ScalarValue, +}; + +use arrow::{ + array::ArrayRef, + datatypes::{Schema, SchemaRef}, + error::{ArrowError, Result as ArrowResult}, + record_batch::RecordBatch, +}; +use log::debug; +use parquet::file::{ + metadata::RowGroupMetaData, + reader::{FileReader, SerializedFileReader}, + statistics::Statistics as ParquetStatistics, +}; + +use fmt::Debug; +use parquet::arrow::{ArrowReader, ParquetFileArrowReader}; + +use tokio::{ + sync::mpsc::{channel, Receiver, Sender}, + task, +}; + +use async_trait::async_trait; + +use crate::datasource::{FilePartition, PartitionedFile}; + +/// Execution plan for scanning one or more Parquet partitions +#[derive(Debug, Clone)] +pub struct ParquetExec { + /// Parquet partitions to read + pub partitions: Vec, + /// Schema after projection is applied + pub schema: SchemaRef, + /// Projection for which columns to load + projection: Vec, + /// Batch size + batch_size: usize, + /// Statistics for the data set (sum of statistics for all partitions) + statistics: Statistics, + /// Execution metrics + metrics: ExecutionPlanMetricsSet, + /// Optional predicate builder + predicate_builder: Option, + /// Optional limit of the number of rows + limit: Option, +} + +/// Represents one partition of a Parquet data set and this currently means one Parquet file. +/// +/// In the future it would be good to support subsets of files based on ranges of row groups +/// so that we can better parallelize reads of large files across available cores (see +/// [ARROW-10995](https://issues.apache.org/jira/browse/ARROW-10995)). +/// +/// We may also want to support reading Parquet files that are partitioned based on a key and +/// in this case we would want this partition struct to represent multiple files for a given +/// partition key (see [ARROW-11019](https://issues.apache.org/jira/browse/ARROW-11019)). +#[derive(Debug, Clone)] +pub struct ParquetPartition { + /// The Parquet filename for this partition + pub file_partition: FilePartition, + /// Execution metrics + metrics: ExecutionPlanMetricsSet, +} + +/// Stores metrics about the parquet execution for a particular parquet file +#[derive(Debug, Clone)] +struct ParquetFileMetrics { + /// Number of times the predicate could not be evaluated + pub predicate_evaluation_errors: metrics::Count, + /// Number of row groups pruned using + pub row_groups_pruned: metrics::Count, +} + +impl ParquetExec { + /// Create a new Parquet reader execution plan provided file list and schema + pub fn try_new( + files: Vec>, + statistics: Statistics, + schema: SchemaRef, + projection: Option>, + predicate: Option, + batch_size: usize, + limit: Option, + ) -> Result { + debug!("Creating ParquetExec, desc: {:?}, projection {:?}, predicate: {:?}, limit: {:?}", + files, projection, predicate, limit); + + let metrics = ExecutionPlanMetricsSet::new(); + + let partitions = files + .into_iter() + .enumerate() + .map(|(i, f)| ParquetPartition::new(f, i, metrics.clone())) + .collect::>(); + + let metrics = ExecutionPlanMetricsSet::new(); + let predicate_creation_errors = + MetricBuilder::new(&metrics).global_counter("num_predicate_creation_errors"); + + let predicate_builder = predicate.and_then(|predicate_expr| { + match PruningPredicate::try_new(&predicate_expr, schema.clone()) { + Ok(predicate_builder) => Some(predicate_builder), + Err(e) => { + debug!( + "Could not create pruning predicate for {:?}: {}", + predicate_expr, e + ); + predicate_creation_errors.add(1); + None + } + } + }); + + Ok(Self::new( + partitions, + schema, + projection, + statistics, + metrics, + predicate_builder, + batch_size, + limit, + )) + } + + /// Create a new Parquet reader execution plan with provided partitions and schema + #[allow(clippy::too_many_arguments)] + fn new( + partitions: Vec, + schema: SchemaRef, + projection: Option>, + statistics: Statistics, + metrics: ExecutionPlanMetricsSet, + predicate_builder: Option, + batch_size: usize, + limit: Option, + ) -> Self { + let projection = match projection { + Some(p) => p, + None => (0..schema.fields().len()).collect(), + }; + + let projected_schema = Schema::new( + projection + .iter() + .map(|i| schema.field(*i).clone()) + .collect(), + ); + + let new_column_statistics = statistics.column_statistics.map(|stats| { + let mut projected_stats = Vec::with_capacity(projection.len()); + for proj in &projection { + projected_stats.push(stats[*proj].clone()); + } + projected_stats + }); + + let statistics = Statistics { + num_rows: statistics.num_rows, + total_byte_size: statistics.total_byte_size, + column_statistics: new_column_statistics, + is_exact: statistics.is_exact, + }; + + Self { + partitions, + schema: Arc::new(projected_schema), + projection, + metrics, + predicate_builder, + batch_size, + statistics, + limit, + } + } + + /// Parquet partitions to read + pub fn partitions(&self) -> &[ParquetPartition] { + &self.partitions + } + + /// Projection for which columns to load + pub fn projection(&self) -> &[usize] { + &self.projection + } + + /// Batch size + pub fn batch_size(&self) -> usize { + self.batch_size + } +} + +impl ParquetPartition { + /// Create a new parquet partition + pub fn new( + files: Vec, + index: usize, + metrics: ExecutionPlanMetricsSet, + ) -> Self { + Self { + file_partition: FilePartition { index, files }, + metrics, + } + } +} + +impl ParquetFileMetrics { + /// Create new metrics + pub fn new( + partition: usize, + filename: &str, + metrics: &ExecutionPlanMetricsSet, + ) -> Self { + let predicate_evaluation_errors = MetricBuilder::new(metrics) + .with_new_label("filename", filename.to_string()) + .counter("predicate_evaluation_errors", partition); + + let row_groups_pruned = MetricBuilder::new(metrics) + .with_new_label("filename", filename.to_string()) + .counter("row_groups_pruned", partition); + + Self { + predicate_evaluation_errors, + row_groups_pruned, + } + } +} + +#[async_trait] +impl ExecutionPlan for ParquetExec { + /// Return a reference to Any that can be used for downcasting + fn as_any(&self) -> &dyn Any { + self + } + + fn schema(&self) -> SchemaRef { + self.schema.clone() + } + + fn children(&self) -> Vec> { + // this is a leaf node and has no children + vec![] + } + + /// Get the output partitioning of this plan + fn output_partitioning(&self) -> Partitioning { + Partitioning::UnknownPartitioning(self.partitions.len()) + } + + fn with_new_children( + &self, + children: Vec>, + ) -> Result> { + if children.is_empty() { + Ok(Arc::new(self.clone())) + } else { + Err(DataFusionError::Internal(format!( + "Children cannot be replaced in {:?}", + self + ))) + } + } + + async fn execute(&self, partition_index: usize) -> Result { + // because the parquet implementation is not thread-safe, it is necessary to execute + // on a thread and communicate with channels + let (response_tx, response_rx): ( + Sender>, + Receiver>, + ) = channel(2); + + let partition = self.partitions[partition_index].clone(); + let metrics = self.metrics.clone(); + let projection = self.projection.clone(); + let predicate_builder = self.predicate_builder.clone(); + let batch_size = self.batch_size; + let limit = self.limit; + + task::spawn_blocking(move || { + if let Err(e) = read_partition( + partition_index, + partition, + metrics, + &projection, + &predicate_builder, + batch_size, + response_tx, + limit, + ) { + println!("Parquet reader thread terminated due to error: {:?}", e); + } + }); + + Ok(RecordBatchReceiverStream::create(&self.schema, response_rx)) + } + + fn fmt_as( + &self, + t: DisplayFormatType, + f: &mut std::fmt::Formatter, + ) -> std::fmt::Result { + match t { + DisplayFormatType::Default => { + let files: Vec<_> = self + .partitions + .iter() + .map(|pp| format!("{}", pp.file_partition)) + .collect(); + + write!( + f, + "ParquetExec: batch_size={}, limit={:?}, partitions=[{}]", + self.batch_size, + self.limit, + files.join(", ") + ) + } + } + } + + fn metrics(&self) -> Option { + Some(self.metrics.clone_inner()) + } + + fn statistics(&self) -> Statistics { + self.statistics.clone() + } +} + +fn send_result( + response_tx: &Sender>, + result: ArrowResult, +) -> Result<()> { + // Note this function is running on its own blockng tokio thread so blocking here is ok. + response_tx + .blocking_send(result) + .map_err(|e| DataFusionError::Execution(e.to_string()))?; + Ok(()) +} + +/// Wraps parquet statistics in a way +/// that implements [`PruningStatistics`] +struct RowGroupPruningStatistics<'a> { + row_group_metadata: &'a [RowGroupMetaData], + parquet_schema: &'a Schema, +} + +/// Extract the min/max statistics from a `ParquetStatistics` object +macro_rules! get_statistic { + ($column_statistics:expr, $func:ident, $bytes_func:ident) => {{ + if !$column_statistics.has_min_max_set() { + return None; + } + match $column_statistics { + ParquetStatistics::Boolean(s) => Some(ScalarValue::Boolean(Some(*s.$func()))), + ParquetStatistics::Int32(s) => Some(ScalarValue::Int32(Some(*s.$func()))), + ParquetStatistics::Int64(s) => Some(ScalarValue::Int64(Some(*s.$func()))), + // 96 bit ints not supported + ParquetStatistics::Int96(_) => None, + ParquetStatistics::Float(s) => Some(ScalarValue::Float32(Some(*s.$func()))), + ParquetStatistics::Double(s) => Some(ScalarValue::Float64(Some(*s.$func()))), + ParquetStatistics::ByteArray(s) => { + let s = std::str::from_utf8(s.$bytes_func()) + .map(|s| s.to_string()) + .ok(); + Some(ScalarValue::Utf8(s)) + } + // type not supported yet + ParquetStatistics::FixedLenByteArray(_) => None, + } + }}; +} + +// Extract the min or max value calling `func` or `bytes_func` on the ParquetStatistics as appropriate +macro_rules! get_min_max_values { + ($self:expr, $column:expr, $func:ident, $bytes_func:ident) => {{ + let (column_index, field) = if let Some((v, f)) = $self.parquet_schema.column_with_name(&$column.name) { + (v, f) + } else { + // Named column was not present + return None + }; + + let data_type = field.data_type(); + let null_scalar: ScalarValue = if let Ok(v) = data_type.try_into() { + v + } else { + // DataFusion doesn't have support for ScalarValues of the column type + return None + }; + + let scalar_values : Vec = $self.row_group_metadata + .iter() + .flat_map(|meta| { + meta.column(column_index).statistics() + }) + .map(|stats| { + get_statistic!(stats, $func, $bytes_func) + }) + .map(|maybe_scalar| { + // column either did't have statistics at all or didn't have min/max values + maybe_scalar.unwrap_or_else(|| null_scalar.clone()) + }) + .collect(); + + // ignore errors converting to arrays (e.g. different types) + ScalarValue::iter_to_array(scalar_values).ok() + }} +} + +impl<'a> PruningStatistics for RowGroupPruningStatistics<'a> { + fn min_values(&self, column: &Column) -> Option { + get_min_max_values!(self, column, min, min_bytes) + } + + fn max_values(&self, column: &Column) -> Option { + get_min_max_values!(self, column, max, max_bytes) + } + + fn num_containers(&self) -> usize { + self.row_group_metadata.len() + } +} + +fn build_row_group_predicate( + predicate_builder: &PruningPredicate, + metrics: ParquetFileMetrics, + row_group_metadata: &[RowGroupMetaData], +) -> Box bool> { + let parquet_schema = predicate_builder.schema().as_ref(); + + let pruning_stats = RowGroupPruningStatistics { + row_group_metadata, + parquet_schema, + }; + let predicate_values = predicate_builder.prune(&pruning_stats); + + match predicate_values { + Ok(values) => { + // NB: false means don't scan row group + let num_pruned = values.iter().filter(|&v| !*v).count(); + metrics.row_groups_pruned.add(num_pruned); + Box::new(move |_, i| values[i]) + } + // stats filter array could not be built + // return a closure which will not filter out any row groups + Err(e) => { + debug!("Error evaluating row group predicate values {}", e); + metrics.predicate_evaluation_errors.add(1); + Box::new(|_r, _i| true) + } + } +} + +#[allow(clippy::too_many_arguments)] +fn read_partition( + partition_index: usize, + partition: ParquetPartition, + metrics: ExecutionPlanMetricsSet, + projection: &[usize], + predicate_builder: &Option, + batch_size: usize, + response_tx: Sender>, + limit: Option, +) -> Result<()> { + 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)?; + if let Some(predicate_builder) = predicate_builder { + let row_group_predicate = build_row_group_predicate( + predicate_builder, + file_metrics, + file_reader.metadata().row_groups(), + ); + file_reader.filter_row_groups(&row_group_predicate); + } + let mut arrow_reader = ParquetFileArrowReader::new(Arc::new(file_reader)); + let mut batch_reader = arrow_reader + .get_record_reader_by_columns(projection.to_owned(), batch_size)?; + loop { + match batch_reader.next() { + Some(Ok(batch)) => { + total_rows += batch.num_rows(); + send_result(&response_tx, Ok(batch))?; + if limit.map(|l| total_rows >= l).unwrap_or(false) { + break 'outer; + } + } + None => { + break; + } + Some(Err(e)) => { + let err_msg = format!( + "Error reading batch from {}: {}", + partitioned_file, + e.to_string() + ); + // send error to operator + send_result( + &response_tx, + Err(ArrowError::ParquetError(err_msg.clone())), + )?; + // terminate thread with error + return Err(DataFusionError::Execution(err_msg)); + } + } + } + } + + // finished reading files (dropping response_tx will close + // channel) + Ok(()) +} + +#[cfg(test)] +mod tests { + use crate::datasource::format::{parquet::ParquetFormat, FileFormat}; + + use super::*; + use arrow::datatypes::{DataType, Field}; + use futures::StreamExt; + use parquet::{ + basic::Type as PhysicalType, + file::{metadata::RowGroupMetaData, statistics::Statistics as ParquetStatistics}, + schema::types::SchemaDescPtr, + }; + + #[tokio::test] + async fn test() -> Result<()> { + let testdata = crate::test_util::parquet_test_data(); + let filename = format!("{}/alltypes_plain.parquet", testdata); + let parquet_exec = ParquetExec::try_new( + vec![vec![PartitionedFile { + path: filename.clone(), + statistics: Statistics::default(), + }]], + Statistics::default(), + ParquetFormat { + enable_pruning: true, + } + .infer_schema(Box::pin(futures::stream::once(async { filename }))) + .await?, + Some(vec![0, 1, 2]), + None, + 1024, + None, + )?; + assert_eq!(parquet_exec.output_partitioning().partition_count(), 1); + + let mut results = parquet_exec.execute(0).await?; + let batch = results.next().await.unwrap()?; + + assert_eq!(8, batch.num_rows()); + assert_eq!(3, batch.num_columns()); + + let schema = batch.schema(); + let field_names: Vec<&str> = + schema.fields().iter().map(|f| f.name().as_str()).collect(); + assert_eq!(vec!["id", "bool_col", "tinyint_col"], field_names); + + let batch = results.next().await; + assert!(batch.is_none()); + + let batch = results.next().await; + assert!(batch.is_none()); + + let batch = results.next().await; + assert!(batch.is_none()); + + Ok(()) + } + + fn parquet_file_metrics() -> ParquetFileMetrics { + let metrics = Arc::new(ExecutionPlanMetricsSet::new()); + ParquetFileMetrics::new(0, "file.parquet", &metrics) + } + + #[test] + fn row_group_predicate_builder_simple_expr() -> Result<()> { + use crate::logical_plan::{col, lit}; + // int > 1 => c1_max > 1 + let expr = col("c1").gt(lit(15)); + let schema = Schema::new(vec![Field::new("c1", DataType::Int32, false)]); + let predicate_builder = PruningPredicate::try_new(&expr, Arc::new(schema))?; + + let schema_descr = get_test_schema_descr(vec![("c1", PhysicalType::INT32)]); + let rgm1 = get_row_group_meta_data( + &schema_descr, + vec![ParquetStatistics::int32(Some(1), Some(10), None, 0, false)], + ); + let rgm2 = get_row_group_meta_data( + &schema_descr, + vec![ParquetStatistics::int32(Some(11), Some(20), None, 0, false)], + ); + let row_group_metadata = vec![rgm1, rgm2]; + let row_group_predicate = build_row_group_predicate( + &predicate_builder, + parquet_file_metrics(), + &row_group_metadata, + ); + let row_group_filter = row_group_metadata + .iter() + .enumerate() + .map(|(i, g)| row_group_predicate(g, i)) + .collect::>(); + assert_eq!(row_group_filter, vec![false, true]); + + Ok(()) + } + + #[test] + fn row_group_predicate_builder_missing_stats() -> Result<()> { + use crate::logical_plan::{col, lit}; + // int > 1 => c1_max > 1 + let expr = col("c1").gt(lit(15)); + let schema = Schema::new(vec![Field::new("c1", DataType::Int32, false)]); + let predicate_builder = PruningPredicate::try_new(&expr, Arc::new(schema))?; + + let schema_descr = get_test_schema_descr(vec![("c1", PhysicalType::INT32)]); + let rgm1 = get_row_group_meta_data( + &schema_descr, + vec![ParquetStatistics::int32(None, None, None, 0, false)], + ); + let rgm2 = get_row_group_meta_data( + &schema_descr, + vec![ParquetStatistics::int32(Some(11), Some(20), None, 0, false)], + ); + let row_group_metadata = vec![rgm1, rgm2]; + let row_group_predicate = build_row_group_predicate( + &predicate_builder, + parquet_file_metrics(), + &row_group_metadata, + ); + let row_group_filter = row_group_metadata + .iter() + .enumerate() + .map(|(i, g)| row_group_predicate(g, i)) + .collect::>(); + // missing statistics for first row group mean that the result from the predicate expression + // is null / undefined so the first row group can't be filtered out + assert_eq!(row_group_filter, vec![true, true]); + + Ok(()) + } + + #[test] + fn row_group_predicate_builder_partial_expr() -> Result<()> { + use crate::logical_plan::{col, lit}; + // test row group predicate with partially supported expression + // int > 1 and int % 2 => c1_max > 1 and true + let expr = col("c1").gt(lit(15)).and(col("c2").modulus(lit(2))); + let schema = Arc::new(Schema::new(vec![ + Field::new("c1", DataType::Int32, false), + Field::new("c2", DataType::Int32, false), + ])); + let predicate_builder = PruningPredicate::try_new(&expr, schema.clone())?; + + let schema_descr = get_test_schema_descr(vec![ + ("c1", PhysicalType::INT32), + ("c2", PhysicalType::INT32), + ]); + let rgm1 = get_row_group_meta_data( + &schema_descr, + vec![ + ParquetStatistics::int32(Some(1), Some(10), None, 0, false), + ParquetStatistics::int32(Some(1), Some(10), None, 0, false), + ], + ); + let rgm2 = get_row_group_meta_data( + &schema_descr, + vec![ + ParquetStatistics::int32(Some(11), Some(20), None, 0, false), + ParquetStatistics::int32(Some(11), Some(20), None, 0, false), + ], + ); + let row_group_metadata = vec![rgm1, rgm2]; + let row_group_predicate = build_row_group_predicate( + &predicate_builder, + parquet_file_metrics(), + &row_group_metadata, + ); + let row_group_filter = row_group_metadata + .iter() + .enumerate() + .map(|(i, g)| row_group_predicate(g, i)) + .collect::>(); + // the first row group is still filtered out because the predicate expression can be partially evaluated + // when conditions are joined using AND + assert_eq!(row_group_filter, vec![false, true]); + + // if conditions in predicate are joined with OR and an unsupported expression is used + // this bypasses the entire predicate expression and no row groups are filtered out + let expr = col("c1").gt(lit(15)).or(col("c2").modulus(lit(2))); + let predicate_builder = PruningPredicate::try_new(&expr, schema)?; + let row_group_predicate = build_row_group_predicate( + &predicate_builder, + parquet_file_metrics(), + &row_group_metadata, + ); + let row_group_filter = row_group_metadata + .iter() + .enumerate() + .map(|(i, g)| row_group_predicate(g, i)) + .collect::>(); + assert_eq!(row_group_filter, vec![true, true]); + + Ok(()) + } + + #[test] + fn row_group_predicate_builder_unsupported_type() -> Result<()> { + use crate::logical_plan::{col, lit}; + // test row group predicate with unsupported statistics type (boolean) + // where a null array is generated for some statistics columns + // int > 1 and bool = true => c1_max > 1 and null + let expr = col("c1").gt(lit(15)).and(col("c2").eq(lit(true))); + let schema = Arc::new(Schema::new(vec![ + Field::new("c1", DataType::Int32, false), + Field::new("c2", DataType::Boolean, false), + ])); + let predicate_builder = PruningPredicate::try_new(&expr, schema)?; + + let schema_descr = get_test_schema_descr(vec![ + ("c1", PhysicalType::INT32), + ("c2", PhysicalType::BOOLEAN), + ]); + let rgm1 = get_row_group_meta_data( + &schema_descr, + vec![ + ParquetStatistics::int32(Some(1), Some(10), None, 0, false), + ParquetStatistics::boolean(Some(false), Some(true), None, 0, false), + ], + ); + let rgm2 = get_row_group_meta_data( + &schema_descr, + vec![ + ParquetStatistics::int32(Some(11), Some(20), None, 0, false), + ParquetStatistics::boolean(Some(false), Some(true), None, 0, false), + ], + ); + let row_group_metadata = vec![rgm1, rgm2]; + let row_group_predicate = build_row_group_predicate( + &predicate_builder, + parquet_file_metrics(), + &row_group_metadata, + ); + let row_group_filter = row_group_metadata + .iter() + .enumerate() + .map(|(i, g)| row_group_predicate(g, i)) + .collect::>(); + // no row group is filtered out because the predicate expression can't be evaluated + // when a null array is generated for a statistics column, + // because the null values propagate to the end result, making the predicate result undefined + assert_eq!(row_group_filter, vec![true, true]); + + Ok(()) + } + + fn get_row_group_meta_data( + schema_descr: &SchemaDescPtr, + column_statistics: Vec, + ) -> RowGroupMetaData { + use parquet::file::metadata::ColumnChunkMetaData; + let mut columns = vec![]; + for (i, s) in column_statistics.iter().enumerate() { + let column = ColumnChunkMetaData::builder(schema_descr.column(i)) + .set_statistics(s.clone()) + .build() + .unwrap(); + columns.push(column); + } + RowGroupMetaData::builder(schema_descr.clone()) + .set_num_rows(1000) + .set_total_byte_size(2000) + .set_column_metadata(columns) + .build() + .unwrap() + } + + fn get_test_schema_descr(fields: Vec<(&str, PhysicalType)>) -> SchemaDescPtr { + use parquet::schema::types::{SchemaDescriptor, Type as SchemaType}; + let mut schema_fields = fields + .iter() + .map(|(n, t)| { + Arc::new(SchemaType::primitive_type_builder(n, *t).build().unwrap()) + }) + .collect::>(); + let schema = SchemaType::group_type_builder("schema") + .with_fields(&mut schema_fields) + .build() + .unwrap(); + + Arc::new(SchemaDescriptor::new(Arc::new(schema))) + } +} diff --git a/datafusion/src/physical_plan/mod.rs b/datafusion/src/physical_plan/mod.rs index fef2af58b99d..0362549874e6 100644 --- a/datafusion/src/physical_plan/mod.rs +++ b/datafusion/src/physical_plan/mod.rs @@ -621,6 +621,7 @@ pub mod empty; pub mod explain; pub mod expressions; pub mod filter; +pub mod format; pub mod functions; pub mod hash_aggregate; pub mod hash_join; diff --git a/datafusion/src/physical_plan/parquet.rs b/datafusion/src/physical_plan/parquet.rs index 80eaf53de5e6..f4ac4c8fddaf 100644 --- a/datafusion/src/physical_plan/parquet.rs +++ b/datafusion/src/physical_plan/parquet.rs @@ -189,57 +189,6 @@ impl ParquetExec { )) } - /// Create a new Parquet reader execution plan provided file list and schema - pub fn try_new_refacto( - files: Vec>, - statistics: Statistics, - schema: SchemaRef, - projection: Option>, - predicate: Option, - batch_size: usize, - limit: Option, - ) -> Result { - debug!("Creating ParquetExec, desc: {:?}, projection {:?}, predicate: {:?}, limit: {:?}", - files, projection, predicate, limit); - - let metrics = ExecutionPlanMetricsSet::new(); - - let partitions = files - .into_iter() - .enumerate() - .map(|(i, f)| ParquetPartition::new(f, i, metrics.clone())) - .collect::>(); - - let metrics = ExecutionPlanMetricsSet::new(); - let predicate_creation_errors = - MetricBuilder::new(&metrics).global_counter("num_predicate_creation_errors"); - - let predicate_builder = predicate.and_then(|predicate_expr| { - match PruningPredicate::try_new(&predicate_expr, schema.clone()) { - Ok(predicate_builder) => Some(predicate_builder), - Err(e) => { - debug!( - "Could not create pruning predicate for {:?}: {}", - predicate_expr, e - ); - predicate_creation_errors.add(1); - None - } - } - }); - - Ok(Self::new( - partitions, - schema, - projection, - statistics, - metrics, - predicate_builder, - batch_size, - limit, - )) - } - /// Create a new Parquet reader execution plan with provided partitions and schema #[allow(clippy::too_many_arguments)] pub fn new( From 522747e83ebb251454d7461690112526678906f5 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 12 Oct 2021 19:24:15 +0200 Subject: [PATCH 08/39] [doc] comments about why we are flattening --- datafusion/src/datasource/format/csv.rs | 1 + datafusion/src/datasource/format/json.rs | 1 + 2 files changed, 2 insertions(+) diff --git a/datafusion/src/datasource/format/csv.rs b/datafusion/src/datasource/format/csv.rs index 0c1db35f1d18..0f2a16ba20a6 100644 --- a/datafusion/src/datasource/format/csv.rs +++ b/datafusion/src/datasource/format/csv.rs @@ -87,6 +87,7 @@ impl FileFormat for CsvFormat { limit: Option, ) -> Result> { let exec = CsvExec::try_new( + // flattening this for now because CsvExec does not support partitioning yet files.into_iter().flatten().map(|f| f.path).collect(), statistics, schema, diff --git a/datafusion/src/datasource/format/json.rs b/datafusion/src/datasource/format/json.rs index 521de52cbf71..88c7631601e9 100644 --- a/datafusion/src/datasource/format/json.rs +++ b/datafusion/src/datasource/format/json.rs @@ -83,6 +83,7 @@ impl FileFormat for JsonFormat { limit: Option, ) -> Result> { let exec = NdJsonExec::try_new( + // flattening this for now because NdJsonExec does not support partitioning yet files.into_iter().flatten().map(|f| f.path).collect(), statistics, schema, From 33c7b914e4b2fecb45ac269a259d3ea9f205697b Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 12 Oct 2021 19:24:15 +0200 Subject: [PATCH 09/39] [test] migrated tests to file formats --- datafusion/src/datasource/format/csv.rs | 119 ++++ datafusion/src/datasource/format/json.rs | 100 ++++ datafusion/src/datasource/format/mod.rs | 5 + datafusion/src/datasource/format/parquet.rs | 622 ++++++++++---------- datafusion/src/logical_plan/expr.rs | 23 + 5 files changed, 554 insertions(+), 315 deletions(-) diff --git a/datafusion/src/datasource/format/csv.rs b/datafusion/src/datasource/format/csv.rs index 0f2a16ba20a6..f5ca43030a4f 100644 --- a/datafusion/src/datasource/format/csv.rs +++ b/datafusion/src/datasource/format/csv.rs @@ -100,3 +100,122 @@ impl FileFormat for CsvFormat { Ok(Arc::new(exec)) } } + +#[cfg(test)] +mod tests { + use arrow::array::StringArray; + + use super::*; + use crate::{datasource::format::string_stream, physical_plan::collect}; + + #[tokio::test] + 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 stream = exec.execute(0).await?; + + let tt_rows: i32 = stream + .map(|batch| { + let batch = batch.unwrap(); + assert_eq!(12, batch.num_columns()); + assert_eq!(2, batch.num_rows()); + }) + .fold(0, |acc, _| async move { acc + 1i32 }) + .await; + + assert_eq!(tt_rows, 50 /* 100/2 */); + + // test metadata + assert_eq!(exec.statistics().num_rows, None); + assert_eq!(exec.statistics().total_byte_size, None); + + Ok(()) + } + + #[tokio::test] + async fn infer_schema() -> Result<()> { + let projection = None; + let exec = get_exec("aggregate_test_100.csv", &projection, 1024).await?; + + let x: Vec = exec + .schema() + .fields() + .iter() + .map(|f| format!("{}: {:?}", f.name(), f.data_type())) + .collect(); + assert_eq!( + vec![ + "c1: Utf8", + "c2: Int64", + "c3: Int64", + "c4: Int64", + "c5: Int64", + "c6: Int64", + "c7: Int64", + "c8: Int64", + "c9: Int64", + "c10: Int64", + "c11: Float64", + "c12: Float64", + "c13: Utf8" + ], + x + ); + + Ok(()) + } + + #[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 batches = collect(exec).await.expect("Collect batches"); + + assert_eq!(1, batches.len()); + assert_eq!(1, batches[0].num_columns()); + assert_eq!(100, batches[0].num_rows()); + + let array = batches[0] + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + let mut values: Vec<&str> = vec![]; + for i in 0..5 { + values.push(array.value(i)); + } + + assert_eq!(vec!["c", "d", "b", "a", "b"], values); + + Ok(()) + } + + async fn get_exec( + file_name: &str, + projection: &Option>, + batch_size: usize, + ) -> Result> { + let testdata = crate::test_util::arrow_test_data(); + let filename = format!("{}/csv/{}", testdata, file_name); + let table = CsvFormat { + has_header: true, + schema_infer_max_rec: Some(1000), + delimiter: b',', + }; + let schema = table + .infer_schema(string_stream(vec![filename.clone()])) + .await + .expect("Schema inference"); + let stats = table.infer_stats(&filename).await.expect("Stats inference"); + let files = vec![vec![PartitionedFile { + path: filename, + statistics: stats.clone(), + }]]; + let exec = table + .create_executor(schema, files, stats, projection, batch_size, &[], None) + .await?; + Ok(exec) + } +} diff --git a/datafusion/src/datasource/format/json.rs b/datafusion/src/datasource/format/json.rs index 88c7631601e9..382f34f30858 100644 --- a/datafusion/src/datasource/format/json.rs +++ b/datafusion/src/datasource/format/json.rs @@ -94,3 +94,103 @@ impl FileFormat for JsonFormat { Ok(Arc::new(exec)) } } + +#[cfg(test)] +mod tests { + use arrow::array::Int64Array; + + use super::*; + use crate::{datasource::format::string_stream, physical_plan::collect}; + + #[tokio::test] + async fn read_small_batches() -> Result<()> { + let projection = None; + let exec = get_exec(&projection, 2).await?; + let stream = exec.execute(0).await?; + + let tt_rows: i32 = stream + .map(|batch| { + let batch = batch.unwrap(); + assert_eq!(4, batch.num_columns()); + assert_eq!(2, batch.num_rows()); + }) + .fold(0, |acc, _| async move { acc + 1i32 }) + .await; + + assert_eq!(tt_rows, 6 /* 12/2 */); + + // test metadata + assert_eq!(exec.statistics().num_rows, None); + assert_eq!(exec.statistics().total_byte_size, None); + + Ok(()) + } + + #[tokio::test] + async fn infer_schema() -> Result<()> { + let projection = None; + let exec = get_exec(&projection, 1024).await?; + + let x: Vec = exec + .schema() + .fields() + .iter() + .map(|f| format!("{}: {:?}", f.name(), f.data_type())) + .collect(); + assert_eq!(vec!["a: Int64", "b: Float64", "c: Boolean", "d: Utf8",], x); + + Ok(()) + } + + #[tokio::test] + async fn read_int_column() -> Result<()> { + let projection = Some(vec![0]); + let exec = get_exec(&projection, 1024).await?; + + let batches = collect(exec).await.expect("Collect batches"); + + assert_eq!(1, batches.len()); + assert_eq!(1, batches[0].num_columns()); + assert_eq!(12, batches[0].num_rows()); + + let array = batches[0] + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + let mut values: Vec = vec![]; + for i in 0..batches[0].num_rows() { + values.push(array.value(i)); + } + + assert_eq!( + vec![1, -10, 2, 1, 7, 1, 1, 5, 1, 1, 1, 100000000000000], + values + ); + + Ok(()) + } + + async fn get_exec( + projection: &Option>, + batch_size: usize, + ) -> Result> { + let filename = "tests/jsons/2.json"; + let table = JsonFormat { + schema_infer_max_rec: Some(1000), + }; + let schema = table + .infer_schema(string_stream(vec![filename.to_owned()])) + .await + .expect("Schema inference"); + let stats = table.infer_stats(filename).await.expect("Stats inference"); + let files = vec![vec![PartitionedFile { + path: filename.to_owned(), + statistics: stats.clone(), + }]]; + let exec = table + .create_executor(schema, files, stats, projection, batch_size, &[], None) + .await?; + Ok(exec) + } +} diff --git a/datafusion/src/datasource/format/mod.rs b/datafusion/src/datasource/format/mod.rs index e4e2c7550575..d4867a8868f8 100644 --- a/datafusion/src/datasource/format/mod.rs +++ b/datafusion/src/datasource/format/mod.rs @@ -38,6 +38,11 @@ use futures::Stream; /// A stream of String that can be used accross await calls pub type StringStream = Pin + Send + Sync>>; +/// Convert a vec into a `StringStream` +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. diff --git a/datafusion/src/datasource/format/parquet.rs b/datafusion/src/datasource/format/parquet.rs index 34f1f1ebdb66..27da05466462 100644 --- a/datafusion/src/datasource/format/parquet.rs +++ b/datafusion/src/datasource/format/parquet.rs @@ -43,6 +43,64 @@ use crate::physical_plan::ExecutionPlan; use crate::physical_plan::{Accumulator, Statistics}; use crate::scalar::ScalarValue; +/// The Apache Parquet `FileFormat` implementation +pub struct ParquetFormat { + /// Activate statistics based row group level pruning + pub enable_pruning: bool, +} + +#[async_trait] +impl FileFormat for ParquetFormat { + async fn infer_schema(&self, mut paths: StringStream) -> 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(Arc::new(schema)) + } + + async fn infer_stats(&self, path: &str) -> Result { + let (_, stats) = fetch_metadata(path)?; + Ok(stats) + } + + async fn create_executor( + &self, + schema: SchemaRef, + files: Vec>, + statistics: Statistics, + projection: &Option>, + batch_size: usize, + filters: &[Expr], + limit: Option, + ) -> 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) + } else { + None + }; + + Ok(Arc::new(ParquetExec::try_new( + files, + statistics, + schema, + projection.clone(), + predicate, + limit + .map(|l| std::cmp::min(l, batch_size)) + .unwrap_or(batch_size), + limit, + )?)) + } +} + fn summarize_min_max( max_values: &mut Vec>, min_values: &mut Vec>, @@ -224,329 +282,263 @@ fn fetch_metadata(path: &str) -> Result<(Schema, Statistics)> { Ok((schema, statistics)) } -/// The Apache Parquet `FileFormat` implementation -pub struct ParquetFormat { - /// Activate statistics based row group level pruning - pub enable_pruning: bool, -} +#[cfg(test)] +mod tests { + use crate::datasource::format::string_stream; + use crate::physical_plan::collect; -#[async_trait] -impl FileFormat for ParquetFormat { - async fn infer_schema(&self, mut paths: StringStream) -> 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(Arc::new(schema)) + use super::*; + use arrow::array::{ + BinaryArray, BooleanArray, Float32Array, Float64Array, Int32Array, + TimestampNanosecondArray, + }; + use futures::StreamExt; + + #[tokio::test] + async fn read_small_batches() -> Result<()> { + let projection = None; + let exec = get_exec("alltypes_plain.parquet", &projection, 2).await?; + let stream = exec.execute(0).await?; + + let _ = stream + .map(|batch| { + let batch = batch.unwrap(); + assert_eq!(11, batch.num_columns()); + assert_eq!(2, batch.num_rows()); + }) + .fold(0, |acc, _| async move { acc + 1i32 }) + .await; + + // test metadata + assert_eq!(exec.statistics().num_rows, Some(8)); + assert_eq!(exec.statistics().total_byte_size, Some(671)); + + Ok(()) } - async fn infer_stats(&self, path: &str) -> Result { - let (_, stats) = fetch_metadata(path)?; - Ok(stats) + #[tokio::test] + async fn read_alltypes_plain_parquet() -> Result<()> { + let projection = None; + let exec = get_exec("alltypes_plain.parquet", &projection, 1024).await?; + + let x: Vec = exec + .schema() + .fields() + .iter() + .map(|f| format!("{}: {:?}", f.name(), f.data_type())) + .collect(); + let y = x.join("\n"); + assert_eq!( + "id: Int32\n\ + bool_col: Boolean\n\ + tinyint_col: Int32\n\ + smallint_col: Int32\n\ + int_col: Int32\n\ + bigint_col: Int64\n\ + float_col: Float32\n\ + double_col: Float64\n\ + date_string_col: Binary\n\ + string_col: Binary\n\ + timestamp_col: Timestamp(Nanosecond, None)", + y + ); + + 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(()) } - async fn create_executor( - &self, - schema: SchemaRef, - files: Vec>, - statistics: Statistics, + #[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 batches = collect(exec).await?; + assert_eq!(1, batches.len()); + assert_eq!(1, batches[0].num_columns()); + assert_eq!(8, batches[0].num_rows()); + + let array = batches[0] + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + let mut values: Vec = vec![]; + for i in 0..batches[0].num_rows() { + values.push(array.value(i)); + } + + assert_eq!( + "[true, false, true, false, true, false, true, false]", + format!("{:?}", values) + ); + + Ok(()) + } + + #[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 batches = collect(exec).await?; + assert_eq!(1, batches.len()); + assert_eq!(1, batches[0].num_columns()); + assert_eq!(8, batches[0].num_rows()); + + let array = batches[0] + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + let mut values: Vec = vec![]; + for i in 0..batches[0].num_rows() { + values.push(array.value(i)); + } + + assert_eq!("[4, 5, 6, 7, 2, 3, 0, 1]", format!("{:?}", values)); + + Ok(()) + } + + #[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 batches = collect(exec).await?; + assert_eq!(1, batches.len()); + assert_eq!(1, batches[0].num_columns()); + assert_eq!(8, batches[0].num_rows()); + + let array = batches[0] + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + let mut values: Vec = vec![]; + for i in 0..batches[0].num_rows() { + values.push(array.value(i)); + } + + assert_eq!("[1235865600000000000, 1235865660000000000, 1238544000000000000, 1238544060000000000, 1233446400000000000, 1233446460000000000, 1230768000000000000, 1230768060000000000]", format!("{:?}", values)); + + Ok(()) + } + + #[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 batches = collect(exec).await?; + assert_eq!(1, batches.len()); + assert_eq!(1, batches[0].num_columns()); + assert_eq!(8, batches[0].num_rows()); + + let array = batches[0] + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + let mut values: Vec = vec![]; + for i in 0..batches[0].num_rows() { + values.push(array.value(i)); + } + + assert_eq!( + "[0.0, 1.1, 0.0, 1.1, 0.0, 1.1, 0.0, 1.1]", + format!("{:?}", values) + ); + + Ok(()) + } + + #[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 batches = collect(exec).await?; + assert_eq!(1, batches.len()); + assert_eq!(1, batches[0].num_columns()); + assert_eq!(8, batches[0].num_rows()); + + let array = batches[0] + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + let mut values: Vec = vec![]; + for i in 0..batches[0].num_rows() { + values.push(array.value(i)); + } + + assert_eq!( + "[0.0, 10.1, 0.0, 10.1, 0.0, 10.1, 0.0, 10.1]", + format!("{:?}", values) + ); + + Ok(()) + } + + #[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 batches = collect(exec).await?; + assert_eq!(1, batches.len()); + assert_eq!(1, batches[0].num_columns()); + assert_eq!(8, batches[0].num_rows()); + + let array = batches[0] + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + let mut values: Vec<&str> = vec![]; + for i in 0..batches[0].num_rows() { + values.push(std::str::from_utf8(array.value(i)).unwrap()); + } + + assert_eq!( + "[\"0\", \"1\", \"0\", \"1\", \"0\", \"1\", \"0\", \"1\"]", + format!("{:?}", values) + ); + + Ok(()) + } + + async fn get_exec( + file_name: &str, projection: &Option>, batch_size: usize, - filters: &[Expr], - limit: Option, ) -> 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) - } else { - None + let testdata = crate::test_util::parquet_test_data(); + let filename = format!("{}/{}", testdata, file_name); + let table = ParquetFormat { + enable_pruning: true, }; - - Ok(Arc::new(ParquetExec::try_new( - files, - statistics, - schema, - projection.clone(), - predicate, - limit - .map(|l| std::cmp::min(l, batch_size)) - .unwrap_or(batch_size), - limit, - )?)) + let schema = table + .infer_schema(string_stream(vec![filename.clone()])) + .await + .expect("Schema inference"); + let stats = table + .infer_stats(&filename.clone()) + .await + .expect("Stats inference"); + let files = vec![vec![PartitionedFile { + path: filename, + statistics: stats.clone(), + }]]; + let exec = table + .create_executor(schema, files, stats, projection, batch_size, &[], None) + .await?; + Ok(exec) } } - -// #[cfg(test)] -// mod tests { -// use super::*; -// use arrow::array::{ -// BinaryArray, BooleanArray, Float32Array, Float64Array, Int32Array, -// TimestampNanosecondArray, -// }; -// use arrow::record_batch::RecordBatch; -// use futures::StreamExt; - -// #[tokio::test] -// async fn read_small_batches() -> Result<()> { -// let table = load_table("alltypes_plain.parquet")?; -// let projection = None; -// let exec = table.scan(&projection, 2, &[], None)?; -// let stream = exec.execute(0).await?; - -// let _ = stream -// .map(|batch| { -// let batch = batch.unwrap(); -// assert_eq!(11, batch.num_columns()); -// assert_eq!(2, batch.num_rows()); -// }) -// .fold(0, |acc, _| async move { acc + 1i32 }) -// .await; - -// // test metadata -// assert_eq!(exec.statistics().num_rows, Some(8)); -// assert_eq!(exec.statistics().total_byte_size, Some(671)); - -// Ok(()) -// } - -// #[tokio::test] -// async fn read_alltypes_plain_parquet() -> Result<()> { -// let table = load_table("alltypes_plain.parquet")?; - -// let x: Vec = table -// .schema() -// .fields() -// .iter() -// .map(|f| format!("{}: {:?}", f.name(), f.data_type())) -// .collect(); -// let y = x.join("\n"); -// assert_eq!( -// "id: Int32\n\ -// bool_col: Boolean\n\ -// tinyint_col: Int32\n\ -// smallint_col: Int32\n\ -// int_col: Int32\n\ -// bigint_col: Int64\n\ -// float_col: Float32\n\ -// double_col: Float64\n\ -// date_string_col: Binary\n\ -// string_col: Binary\n\ -// timestamp_col: Timestamp(Nanosecond, None)", -// y -// ); - -// let projection = None; -// let batch = get_first_batch(table, &projection).await?; - -// assert_eq!(11, batch.num_columns()); -// assert_eq!(8, batch.num_rows()); - -// Ok(()) -// } - -// #[tokio::test] -// async fn read_bool_alltypes_plain_parquet() -> Result<()> { -// let table = load_table("alltypes_plain.parquet")?; -// let projection = Some(vec![1]); -// let batch = get_first_batch(table, &projection).await?; - -// assert_eq!(1, batch.num_columns()); -// assert_eq!(8, batch.num_rows()); - -// let array = batch -// .column(0) -// .as_any() -// .downcast_ref::() -// .unwrap(); -// let mut values: Vec = vec![]; -// for i in 0..batch.num_rows() { -// values.push(array.value(i)); -// } - -// assert_eq!( -// "[true, false, true, false, true, false, true, false]", -// format!("{:?}", values) -// ); - -// Ok(()) -// } - -// #[tokio::test] -// async fn read_i32_alltypes_plain_parquet() -> Result<()> { -// let table = load_table("alltypes_plain.parquet")?; -// let projection = Some(vec![0]); -// let batch = get_first_batch(table, &projection).await?; - -// assert_eq!(1, batch.num_columns()); -// assert_eq!(8, batch.num_rows()); - -// let array = batch -// .column(0) -// .as_any() -// .downcast_ref::() -// .unwrap(); -// let mut values: Vec = vec![]; -// for i in 0..batch.num_rows() { -// values.push(array.value(i)); -// } - -// assert_eq!("[4, 5, 6, 7, 2, 3, 0, 1]", format!("{:?}", values)); - -// Ok(()) -// } - -// #[tokio::test] -// async fn read_i96_alltypes_plain_parquet() -> Result<()> { -// let table = load_table("alltypes_plain.parquet")?; -// let projection = Some(vec![10]); -// let batch = get_first_batch(table, &projection).await?; - -// assert_eq!(1, batch.num_columns()); -// assert_eq!(8, batch.num_rows()); - -// let array = batch -// .column(0) -// .as_any() -// .downcast_ref::() -// .unwrap(); -// let mut values: Vec = vec![]; -// for i in 0..batch.num_rows() { -// values.push(array.value(i)); -// } - -// assert_eq!("[1235865600000000000, 1235865660000000000, 1238544000000000000, 1238544060000000000, 1233446400000000000, 1233446460000000000, 1230768000000000000, 1230768060000000000]", format!("{:?}", values)); - -// Ok(()) -// } - -// #[tokio::test] -// async fn read_f32_alltypes_plain_parquet() -> Result<()> { -// let table = load_table("alltypes_plain.parquet")?; -// let projection = Some(vec![6]); -// let batch = get_first_batch(table, &projection).await?; - -// assert_eq!(1, batch.num_columns()); -// assert_eq!(8, batch.num_rows()); - -// let array = batch -// .column(0) -// .as_any() -// .downcast_ref::() -// .unwrap(); -// let mut values: Vec = vec![]; -// for i in 0..batch.num_rows() { -// values.push(array.value(i)); -// } - -// assert_eq!( -// "[0.0, 1.1, 0.0, 1.1, 0.0, 1.1, 0.0, 1.1]", -// format!("{:?}", values) -// ); - -// Ok(()) -// } - -// #[tokio::test] -// async fn read_f64_alltypes_plain_parquet() -> Result<()> { -// let table = load_table("alltypes_plain.parquet")?; -// let projection = Some(vec![7]); -// let batch = get_first_batch(table, &projection).await?; - -// assert_eq!(1, batch.num_columns()); -// assert_eq!(8, batch.num_rows()); - -// let array = batch -// .column(0) -// .as_any() -// .downcast_ref::() -// .unwrap(); -// let mut values: Vec = vec![]; -// for i in 0..batch.num_rows() { -// values.push(array.value(i)); -// } - -// assert_eq!( -// "[0.0, 10.1, 0.0, 10.1, 0.0, 10.1, 0.0, 10.1]", -// format!("{:?}", values) -// ); - -// Ok(()) -// } - -// #[tokio::test] -// async fn read_binary_alltypes_plain_parquet() -> Result<()> { -// let table = load_table("alltypes_plain.parquet")?; -// let projection = Some(vec![9]); -// let batch = get_first_batch(table, &projection).await?; - -// assert_eq!(1, batch.num_columns()); -// assert_eq!(8, batch.num_rows()); - -// let array = batch -// .column(0) -// .as_any() -// .downcast_ref::() -// .unwrap(); -// let mut values: Vec<&str> = vec![]; -// for i in 0..batch.num_rows() { -// values.push(std::str::from_utf8(array.value(i)).unwrap()); -// } - -// assert_eq!( -// "[\"0\", \"1\", \"0\", \"1\", \"0\", \"1\", \"0\", \"1\"]", -// format!("{:?}", values) -// ); - -// Ok(()) -// } - -// fn load_table(name: &str) -> Result> { -// let testdata = crate::test_util::parquet_test_data(); -// let filename = format!("{}/{}", testdata, name); -// let table = ParquetTable::try_new(&filename, 2)?; -// Ok(Arc::new(table)) -// } - -// async fn get_first_batch( -// table: Arc, -// projection: &Option>, -// ) -> Result { -// let exec = table.scan(projection, 1024, &[], None)?; -// let mut it = exec.execute(0).await?; -// it.next() -// .await -// .expect("should have received at least one batch") -// .map_err(|e| e.into()) -// } - -// #[test] -// fn combine_zero_filters() { -// let result = combine_filters(&[]); -// assert_eq!(result, None); -// } - -// #[test] -// fn combine_one_filter() { -// use crate::logical_plan::{binary_expr, col, lit, Operator}; -// let filter = binary_expr(col("c1"), Operator::Lt, lit(1)); -// let result = combine_filters(&[filter.clone()]); -// assert_eq!(result, Some(filter)); -// } - -// #[test] -// fn combine_multiple_filters() { -// use crate::logical_plan::{and, binary_expr, col, lit, Operator}; -// let filter1 = binary_expr(col("c1"), Operator::Lt, lit(1)); -// let filter2 = binary_expr(col("c2"), Operator::Lt, lit(2)); -// let filter3 = binary_expr(col("c3"), Operator::Lt, lit(3)); -// let result = -// combine_filters(&[filter1.clone(), filter2.clone(), filter3.clone()]); -// assert_eq!(result, Some(and(and(filter1, filter2), filter3))); -// } -// } diff --git a/datafusion/src/logical_plan/expr.rs b/datafusion/src/logical_plan/expr.rs index 8ef69e9b0cfe..d50d5331ff1a 100644 --- a/datafusion/src/logical_plan/expr.rs +++ b/datafusion/src/logical_plan/expr.rs @@ -2221,4 +2221,27 @@ mod tests { assert!(exp2 < exp3); assert!(exp3 > exp2); } + + #[test] + fn combine_zero_filters() { + let result = combine_filters(&[]); + assert_eq!(result, None); + } + + #[test] + fn combine_one_filter() { + let filter = binary_expr(col("c1"), Operator::Lt, lit(1)); + let result = combine_filters(&[filter.clone()]); + assert_eq!(result, Some(filter)); + } + + #[test] + fn combine_multiple_filters() { + let filter1 = binary_expr(col("c1"), Operator::Lt, lit(1)); + let filter2 = binary_expr(col("c2"), Operator::Lt, lit(2)); + let filter3 = binary_expr(col("c3"), Operator::Lt, lit(3)); + let result = + combine_filters(&[filter1.clone(), filter2.clone(), filter3.clone()]); + assert_eq!(result, Some(and(and(filter1, filter2), filter3))); + } } From 0bc191e99eeb089fc1644ba1be484543dc1a8ec0 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 12 Oct 2021 19:24:15 +0200 Subject: [PATCH 10/39] [test] improve listing test --- datafusion/src/datasource/listing.rs | 23 ++++++++++------------- 1 file changed, 10 insertions(+), 13 deletions(-) diff --git a/datafusion/src/datasource/listing.rs b/datafusion/src/datasource/listing.rs index a33f47b64a17..a4c5a2b1bb66 100644 --- a/datafusion/src/datasource/listing.rs +++ b/datafusion/src/datasource/listing.rs @@ -212,7 +212,6 @@ fn split_files( #[cfg(test)] mod tests { use super::*; - use futures::StreamExt; #[test] fn test_split_files() { @@ -251,20 +250,16 @@ mod tests { } #[tokio::test] - async fn read_small_batches() -> Result<()> { + async fn read_single_file() -> Result<()> { let table = load_table("alltypes_plain.parquet").await?; let projection = None; - let exec = table.scan(&projection, 2, &[], None).await?; - let stream = exec.execute(0).await?; + let exec = table + .scan(&projection, 1024, &[], None) + .await + .expect("Scan table"); - let _ = stream - .map(|batch| { - let batch = batch.unwrap(); - assert_eq!(11, batch.num_columns()); - assert_eq!(2, batch.num_rows()); - }) - .fold(0, |acc, _| async move { acc + 1i32 }) - .await; + assert_eq!(exec.children().len(), 0); + assert_eq!(exec.output_partitioning().partition_count(), 1); // test metadata assert_eq!(exec.statistics().num_rows, Some(8)); @@ -273,6 +268,8 @@ mod tests { Ok(()) } + // TODO add tests on listing once the ObjectStore abstraction is added + async fn load_table(name: &str) -> Result> { let testdata = crate::test_util::parquet_test_data(); let filename = format!("{}/{}", testdata, name); @@ -286,7 +283,7 @@ mod tests { collect_stat: true, }; // here we resolve the schema locally - let schema = opt.infer_schema(&filename).await?; + let schema = opt.infer_schema(&filename).await.expect("Infer schema"); let table = ListingTable::try_new(&filename, schema, opt)?; Ok(Arc::new(table)) } From fa115db8e2e3b8643d6ab7dd621f2bf816e8d5c7 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 12 Oct 2021 19:24:15 +0200 Subject: [PATCH 11/39] [feat] add avro to refactored format providers --- datafusion/src/datasource/format/avro.rs | 373 ++++++++++++++++++++ datafusion/src/datasource/format/csv.rs | 11 +- datafusion/src/datasource/format/json.rs | 8 +- datafusion/src/datasource/format/mod.rs | 1 + datafusion/src/datasource/format/parquet.rs | 8 +- datafusion/src/physical_plan/format/avro.rs | 276 +++++++++++++++ datafusion/src/physical_plan/format/csv.rs | 2 +- datafusion/src/physical_plan/format/json.rs | 2 +- datafusion/src/physical_plan/format/mod.rs | 2 + 9 files changed, 669 insertions(+), 14 deletions(-) create mode 100644 datafusion/src/datasource/format/avro.rs create mode 100644 datafusion/src/physical_plan/format/avro.rs diff --git a/datafusion/src/datasource/format/avro.rs b/datafusion/src/datasource/format/avro.rs new file mode 100644 index 000000000000..d785361af10b --- /dev/null +++ b/datafusion/src/datasource/format/avro.rs @@ -0,0 +1,373 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! Apache Avro format abstractions + +use std::sync::Arc; + +use arrow::datatypes::Schema; +use arrow::{self, datatypes::SchemaRef}; +use async_trait::async_trait; +use futures::StreamExt; +use std::fs::File; + +use super::{FileFormat, StringStream}; +use crate::avro_to_arrow::read_avro_schema_from_reader; +use crate::datasource::PartitionedFile; +use crate::error::Result; +use crate::logical_plan::Expr; +use crate::physical_plan::format::AvroExec; +use crate::physical_plan::ExecutionPlan; +use crate::physical_plan::Statistics; + +/// Line-delimited Avro `FileFormat` implementation. +pub struct AvroFormat {} + +#[async_trait] +impl FileFormat for AvroFormat { + async fn infer_schema(&self, mut paths: StringStream) -> 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)?; + schemas.push(schema); + } + let merged_schema = Schema::try_merge(schemas)?; + Ok(Arc::new(merged_schema)) + } + + async fn infer_stats(&self, _path: &str) -> Result { + Ok(Statistics::default()) + } + + async fn create_executor( + &self, + schema: SchemaRef, + files: Vec>, + statistics: Statistics, + projection: &Option>, + batch_size: usize, + _filters: &[Expr], + limit: Option, + ) -> Result> { + let exec = AvroExec::try_new( + // flattening this for now because CsvExec does not support partitioning yet + files.into_iter().flatten().map(|f| f.path).collect(), + statistics, + schema, + projection.clone(), + batch_size, + limit, + )?; + Ok(Arc::new(exec)) + } +} + +#[cfg(test)] +#[cfg(feature = "avro")] +mod tests { + use crate::datasource::format::string_stream; + use crate::physical_plan::collect; + + use super::*; + use arrow::array::{ + BinaryArray, BooleanArray, Float32Array, Float64Array, Int32Array, + TimestampMicrosecondArray, + }; + use futures::StreamExt; + + #[tokio::test] + async fn read_small_batches() -> Result<()> { + let projection = None; + let exec = get_exec("alltypes_plain.avro", &projection, 2).await?; + let stream = exec.execute(0).await?; + + let _ = stream + .map(|batch| { + let batch = batch.unwrap(); + assert_eq!(11, batch.num_columns()); + assert_eq!(2, batch.num_rows()); + }) + .fold(0, |acc, _| async move { acc + 1i32 }) + .await; + + Ok(()) + } + + #[tokio::test] + async fn read_alltypes_plain_avro() -> Result<()> { + let projection = None; + let exec = get_exec("alltypes_plain.avro", &projection, 1024).await?; + + let x: Vec = exec + .schema() + .fields() + .iter() + .map(|f| format!("{}: {:?}", f.name(), f.data_type())) + .collect(); + assert_eq!( + vec![ + "id: Int32", + "bool_col: Boolean", + "tinyint_col: Int32", + "smallint_col: Int32", + "int_col: Int32", + "bigint_col: Int64", + "float_col: Float32", + "double_col: Float64", + "date_string_col: Binary", + "string_col: Binary", + "timestamp_col: Timestamp(Microsecond, None)", + ], + x + ); + + let batches = collect(exec).await?; + assert_eq!(batches.len(), 1); + + let expected = vec![ + "+----+----------+-------------+--------------+---------+------------+-----------+------------+------------------+------------+---------------------+", + "| id | bool_col | tinyint_col | smallint_col | int_col | bigint_col | float_col | double_col | date_string_col | string_col | timestamp_col |", + "+----+----------+-------------+--------------+---------+------------+-----------+------------+------------------+------------+---------------------+", + "| 4 | true | 0 | 0 | 0 | 0 | 0 | 0 | 30332f30312f3039 | 30 | 2009-03-01 00:00:00 |", + "| 5 | false | 1 | 1 | 1 | 10 | 1.1 | 10.1 | 30332f30312f3039 | 31 | 2009-03-01 00:01:00 |", + "| 6 | true | 0 | 0 | 0 | 0 | 0 | 0 | 30342f30312f3039 | 30 | 2009-04-01 00:00:00 |", + "| 7 | false | 1 | 1 | 1 | 10 | 1.1 | 10.1 | 30342f30312f3039 | 31 | 2009-04-01 00:01:00 |", + "| 2 | true | 0 | 0 | 0 | 0 | 0 | 0 | 30322f30312f3039 | 30 | 2009-02-01 00:00:00 |", + "| 3 | false | 1 | 1 | 1 | 10 | 1.1 | 10.1 | 30322f30312f3039 | 31 | 2009-02-01 00:01:00 |", + "| 0 | true | 0 | 0 | 0 | 0 | 0 | 0 | 30312f30312f3039 | 30 | 2009-01-01 00:00:00 |", + "| 1 | false | 1 | 1 | 1 | 10 | 1.1 | 10.1 | 30312f30312f3039 | 31 | 2009-01-01 00:01:00 |", + "+----+----------+-------------+--------------+---------+------------+-----------+------------+------------------+------------+---------------------+", + ]; + + crate::assert_batches_eq!(expected, &batches); + Ok(()) + } + + #[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 batches = collect(exec).await?; + assert_eq!(batches.len(), 1); + assert_eq!(1, batches[0].num_columns()); + assert_eq!(8, batches[0].num_rows()); + + let array = batches[0] + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + let mut values: Vec = vec![]; + for i in 0..batches[0].num_rows() { + values.push(array.value(i)); + } + + assert_eq!( + "[true, false, true, false, true, false, true, false]", + format!("{:?}", values) + ); + + Ok(()) + } + + #[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 batches = collect(exec).await?; + assert_eq!(batches.len(), 1); + assert_eq!(1, batches[0].num_columns()); + assert_eq!(8, batches[0].num_rows()); + + let array = batches[0] + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + let mut values: Vec = vec![]; + for i in 0..batches[0].num_rows() { + values.push(array.value(i)); + } + + assert_eq!("[4, 5, 6, 7, 2, 3, 0, 1]", format!("{:?}", values)); + + Ok(()) + } + + #[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 batches = collect(exec).await?; + assert_eq!(batches.len(), 1); + assert_eq!(1, batches[0].num_columns()); + assert_eq!(8, batches[0].num_rows()); + + let array = batches[0] + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + let mut values: Vec = vec![]; + for i in 0..batches[0].num_rows() { + values.push(array.value(i)); + } + + assert_eq!("[1235865600000000, 1235865660000000, 1238544000000000, 1238544060000000, 1233446400000000, 1233446460000000, 1230768000000000, 1230768060000000]", format!("{:?}", values)); + + Ok(()) + } + + #[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 batches = collect(exec).await?; + assert_eq!(batches.len(), 1); + assert_eq!(1, batches[0].num_columns()); + assert_eq!(8, batches[0].num_rows()); + + let array = batches[0] + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + let mut values: Vec = vec![]; + for i in 0..batches[0].num_rows() { + values.push(array.value(i)); + } + + assert_eq!( + "[0.0, 1.1, 0.0, 1.1, 0.0, 1.1, 0.0, 1.1]", + format!("{:?}", values) + ); + + Ok(()) + } + + #[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 batches = collect(exec).await?; + assert_eq!(batches.len(), 1); + assert_eq!(1, batches[0].num_columns()); + assert_eq!(8, batches[0].num_rows()); + + let array = batches[0] + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + let mut values: Vec = vec![]; + for i in 0..batches[0].num_rows() { + values.push(array.value(i)); + } + + assert_eq!( + "[0.0, 10.1, 0.0, 10.1, 0.0, 10.1, 0.0, 10.1]", + format!("{:?}", values) + ); + + Ok(()) + } + + #[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 batches = collect(exec).await?; + assert_eq!(batches.len(), 1); + assert_eq!(1, batches[0].num_columns()); + assert_eq!(8, batches[0].num_rows()); + + let array = batches[0] + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + let mut values: Vec<&str> = vec![]; + for i in 0..batches[0].num_rows() { + values.push(std::str::from_utf8(array.value(i)).unwrap()); + } + + assert_eq!( + "[\"0\", \"1\", \"0\", \"1\", \"0\", \"1\", \"0\", \"1\"]", + format!("{:?}", values) + ); + + Ok(()) + } + + async fn get_exec( + file_name: &str, + projection: &Option>, + batch_size: usize, + ) -> Result> { + let testdata = crate::test_util::arrow_test_data(); + let filename = format!("{}/avro/{}", testdata, file_name); + let format = AvroFormat {}; + let schema = format + .infer_schema(string_stream(vec![filename.clone()])) + .await + .expect("Schema inference"); + let stats = format + .infer_stats(&filename) + .await + .expect("Stats inference"); + let files = vec![vec![PartitionedFile { + path: filename, + statistics: stats.clone(), + }]]; + let exec = format + .create_executor(schema, files, stats, projection, batch_size, &[], None) + .await?; + Ok(exec) + } +} + +#[cfg(test)] +#[cfg(not(feature = "avro"))] +mod tests { + use super::*; + + use crate::datasource::format::string_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 {} + .infer_schema(string_stream(vec![filename])) + .await; + assert!(matches!( + schema_result, + Err(DataFusionError::NotImplemented(msg)) + if msg == *"cannot read avro schema without the 'avro' feature enabled" + )); + + Ok(()) + } +} diff --git a/datafusion/src/datasource/format/csv.rs b/datafusion/src/datasource/format/csv.rs index f5ca43030a4f..4b3e229f3309 100644 --- a/datafusion/src/datasource/format/csv.rs +++ b/datafusion/src/datasource/format/csv.rs @@ -199,21 +199,24 @@ mod tests { ) -> Result> { let testdata = crate::test_util::arrow_test_data(); let filename = format!("{}/csv/{}", testdata, file_name); - let table = CsvFormat { + let format = CsvFormat { has_header: true, schema_infer_max_rec: Some(1000), delimiter: b',', }; - let schema = table + let schema = format .infer_schema(string_stream(vec![filename.clone()])) .await .expect("Schema inference"); - let stats = table.infer_stats(&filename).await.expect("Stats inference"); + let stats = format + .infer_stats(&filename) + .await + .expect("Stats inference"); let files = vec![vec![PartitionedFile { path: filename, statistics: stats.clone(), }]]; - let exec = table + let exec = format .create_executor(schema, files, stats, projection, batch_size, &[], None) .await?; Ok(exec) diff --git a/datafusion/src/datasource/format/json.rs b/datafusion/src/datasource/format/json.rs index 382f34f30858..c2b9628423d8 100644 --- a/datafusion/src/datasource/format/json.rs +++ b/datafusion/src/datasource/format/json.rs @@ -176,19 +176,19 @@ mod tests { batch_size: usize, ) -> Result> { let filename = "tests/jsons/2.json"; - let table = JsonFormat { + let format = JsonFormat { schema_infer_max_rec: Some(1000), }; - let schema = table + let schema = format .infer_schema(string_stream(vec![filename.to_owned()])) .await .expect("Schema inference"); - let stats = table.infer_stats(filename).await.expect("Stats inference"); + let stats = format.infer_stats(filename).await.expect("Stats inference"); let files = vec![vec![PartitionedFile { path: filename.to_owned(), statistics: stats.clone(), }]]; - let exec = table + let exec = format .create_executor(schema, files, stats, projection, batch_size, &[], None) .await?; Ok(exec) diff --git a/datafusion/src/datasource/format/mod.rs b/datafusion/src/datasource/format/mod.rs index d4867a8868f8..a6143353e6b1 100644 --- a/datafusion/src/datasource/format/mod.rs +++ b/datafusion/src/datasource/format/mod.rs @@ -17,6 +17,7 @@ //! Module containing helper methods for the various file formats +pub mod avro; pub mod csv; pub mod json; pub mod parquet; diff --git a/datafusion/src/datasource/format/parquet.rs b/datafusion/src/datasource/format/parquet.rs index 27da05466462..8376f9936f27 100644 --- a/datafusion/src/datasource/format/parquet.rs +++ b/datafusion/src/datasource/format/parquet.rs @@ -521,14 +521,14 @@ mod tests { ) -> Result> { let testdata = crate::test_util::parquet_test_data(); let filename = format!("{}/{}", testdata, file_name); - let table = ParquetFormat { + let format = ParquetFormat { enable_pruning: true, }; - let schema = table + let schema = format .infer_schema(string_stream(vec![filename.clone()])) .await .expect("Schema inference"); - let stats = table + let stats = format .infer_stats(&filename.clone()) .await .expect("Stats inference"); @@ -536,7 +536,7 @@ mod tests { path: filename, statistics: stats.clone(), }]]; - let exec = table + let exec = format .create_executor(schema, files, stats, projection, batch_size, &[], None) .await?; Ok(exec) diff --git a/datafusion/src/physical_plan/format/avro.rs b/datafusion/src/physical_plan/format/avro.rs new file mode 100644 index 000000000000..4163500bd393 --- /dev/null +++ b/datafusion/src/physical_plan/format/avro.rs @@ -0,0 +1,276 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! Execution plan for reading line-delimited Avro files +use crate::error::{DataFusionError, Result}; +#[cfg(feature = "avro")] +use crate::physical_plan::RecordBatchStream; +use crate::physical_plan::{ + DisplayFormatType, ExecutionPlan, Partitioning, SendableRecordBatchStream, Statistics, +}; +use arrow::datatypes::{Schema, SchemaRef}; +#[cfg(feature = "avro")] +use arrow::{error::Result as ArrowResult, record_batch::RecordBatch}; +use async_trait::async_trait; +#[cfg(feature = "avro")] +use futures::Stream; +use std::any::Any; +use std::sync::Arc; +#[cfg(feature = "avro")] +use std::{ + fs::File, + io::Read, + pin::Pin, + task::{Context, Poll}, +}; + +/// Execution plan for scanning Avro data source +#[derive(Debug, Clone)] +pub struct AvroExec { + files: Vec, + statistics: Statistics, + schema: SchemaRef, + projection: Option>, + projected_schema: SchemaRef, + batch_size: usize, + limit: Option, +} + +impl AvroExec { + /// Create a new JSON reader execution plan provided file list and schema + /// TODO: support partitiond file list (Vec>) + pub fn try_new( + files: Vec, + statistics: Statistics, + schema: SchemaRef, + projection: Option>, + batch_size: usize, + limit: Option, + ) -> Result { + let projected_schema = match &projection { + None => Arc::clone(&schema), + Some(p) => Arc::new(Schema::new( + p.iter().map(|i| schema.field(*i).clone()).collect(), + )), + }; + + Ok(Self { + files, + statistics, + schema, + projection, + projected_schema, + batch_size, + limit, + }) + } +} + +#[async_trait] +impl ExecutionPlan for AvroExec { + fn as_any(&self) -> &dyn Any { + self + } + + fn schema(&self) -> SchemaRef { + self.projected_schema.clone() + } + + fn output_partitioning(&self) -> Partitioning { + Partitioning::UnknownPartitioning(self.files.len()) + } + + fn children(&self) -> Vec> { + Vec::new() + } + + fn with_new_children( + &self, + children: Vec>, + ) -> Result> { + if children.is_empty() { + Ok(Arc::new(self.clone())) + } else { + Err(DataFusionError::Internal(format!( + "Children cannot be replaced in {:?}", + self + ))) + } + } + + #[cfg(not(feature = "avro"))] + async fn execute(&self, _partition: usize) -> Result { + Err(DataFusionError::NotImplemented( + "Cannot execute avro plan without avro feature enabled".to_string(), + )) + } + + #[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))) + } + + fn fmt_as( + &self, + t: DisplayFormatType, + f: &mut std::fmt::Formatter, + ) -> std::fmt::Result { + match t { + DisplayFormatType::Default => { + write!( + f, + "AvroExec: batch_size={}, limit={:?}, partitions=[{}]", + self.batch_size, + self.limit, + self.files.join(", ") + ) + } + } + } + + fn statistics(&self) -> Statistics { + self.statistics.clone() + } +} + +#[cfg(feature = "avro")] +struct AvroStream<'a, R: Read> { + reader: crate::avro_to_arrow::Reader<'a, R>, + remain: Option, +} + +#[cfg(feature = "avro")] +impl<'a, R: Read> AvroStream<'a, R> { + fn new(reader: crate::avro_to_arrow::Reader<'a, R>, limit: Option) -> Self { + Self { + reader, + remain: limit, + } + } +} + +#[cfg(feature = "avro")] +impl Stream for AvroStream<'_, R> { + type Item = ArrowResult; + + fn poll_next( + mut self: Pin<&mut Self>, + _cx: &mut Context<'_>, + ) -> Poll> { + if let Some(remain) = self.remain.as_mut() { + if *remain < 1 { + return Poll::Ready(None); + } + } + + Poll::Ready(match self.reader.next() { + Ok(Some(item)) => { + if let Some(remain) = self.remain.as_mut() { + if *remain >= item.num_rows() { + *remain -= item.num_rows(); + Some(Ok(item)) + } else { + let len = *remain; + *remain = 0; + Some(Ok(RecordBatch::try_new( + item.schema(), + item.columns() + .iter() + .map(|column| column.slice(0, len)) + .collect(), + )?)) + } + } else { + Some(Ok(item)) + } + } + Ok(None) => None, + Err(err) => Some(Err(err)), + }) + } +} + +#[cfg(feature = "avro")] +impl RecordBatchStream for AvroStream<'_, R> { + fn schema(&self) -> SchemaRef { + self.reader.schema() + } +} + +#[cfg(test)] +#[cfg(feature = "avro")] +mod tests { + + use super::*; + + #[tokio::test] + async fn test() -> Result<()> { + use futures::StreamExt; + + use crate::datasource::format::{avro::AvroFormat, FileFormat}; + + let testdata = crate::test_util::arrow_test_data(); + let filename = format!("{}/avro/alltypes_plain.avro", testdata); + let avro_exec = AvroExec::try_new( + vec![filename.clone()], + Statistics::default(), + AvroFormat {} + .infer_schema(Box::pin(futures::stream::once(async { filename }))) + .await?, + Some(vec![0, 1, 2]), + 1024, + None, + )?; + assert_eq!(avro_exec.output_partitioning().partition_count(), 1); + + let mut results = avro_exec.execute(0).await?; + let batch = results.next().await.unwrap()?; + + assert_eq!(8, batch.num_rows()); + assert_eq!(3, batch.num_columns()); + + let schema = batch.schema(); + let field_names: Vec<&str> = + schema.fields().iter().map(|f| f.name().as_str()).collect(); + assert_eq!(vec!["id", "bool_col", "tinyint_col"], field_names); + + let batch = results.next().await; + assert!(batch.is_none()); + + let batch = results.next().await; + assert!(batch.is_none()); + + let batch = results.next().await; + assert!(batch.is_none()); + + Ok(()) + } +} diff --git a/datafusion/src/physical_plan/format/csv.rs b/datafusion/src/physical_plan/format/csv.rs index 0ed26f463152..cb4013af6813 100644 --- a/datafusion/src/physical_plan/format/csv.rs +++ b/datafusion/src/physical_plan/format/csv.rs @@ -61,7 +61,7 @@ pub struct CsvExec { impl CsvExec { /// Create a new CSV reader execution plan provided file list and schema - /// TODO: support partitiondd file list (Vec>) + /// TODO: support partitiond file list (Vec>) #[allow(clippy::too_many_arguments)] pub fn try_new( files: Vec, diff --git a/datafusion/src/physical_plan/format/json.rs b/datafusion/src/physical_plan/format/json.rs index 0dbcc522f937..5ab94b142416 100644 --- a/datafusion/src/physical_plan/format/json.rs +++ b/datafusion/src/physical_plan/format/json.rs @@ -53,7 +53,7 @@ pub struct NdJsonExec { impl NdJsonExec { /// Create a new JSON reader execution plan provided file list and schema - /// TODO: support partitiondd file list (Vec>) + /// TODO: support partitiond file list (Vec>) pub fn try_new( files: Vec, statistics: Statistics, diff --git a/datafusion/src/physical_plan/format/mod.rs b/datafusion/src/physical_plan/format/mod.rs index cd6cebe60338..aa9359c30da4 100644 --- a/datafusion/src/physical_plan/format/mod.rs +++ b/datafusion/src/physical_plan/format/mod.rs @@ -17,10 +17,12 @@ //! Execution plans that read file formats +mod avro; mod csv; mod json; mod parquet; pub use self::parquet::ParquetExec; +pub use avro::AvroExec; pub use csv::CsvExec; pub use json::NdJsonExec; From 553097ed2991dd8ec4798103a9197066561f56f6 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 12 Oct 2021 19:24:15 +0200 Subject: [PATCH 12/39] [fix] remove try from new when unnecessary --- datafusion/src/datasource/format/avro.rs | 4 +- datafusion/src/datasource/format/csv.rs | 4 +- datafusion/src/datasource/format/json.rs | 4 +- datafusion/src/datasource/format/parquet.rs | 4 +- datafusion/src/physical_plan/format/avro.rs | 12 ++-- datafusion/src/physical_plan/format/csv.rs | 16 ++--- datafusion/src/physical_plan/format/json.rs | 16 ++--- .../src/physical_plan/format/parquet.rs | 70 ++++++------------- 8 files changed, 51 insertions(+), 79 deletions(-) diff --git a/datafusion/src/datasource/format/avro.rs b/datafusion/src/datasource/format/avro.rs index d785361af10b..52f52d1d6d8c 100644 --- a/datafusion/src/datasource/format/avro.rs +++ b/datafusion/src/datasource/format/avro.rs @@ -64,7 +64,7 @@ impl FileFormat for AvroFormat { _filters: &[Expr], limit: Option, ) -> Result> { - let exec = AvroExec::try_new( + let exec = AvroExec::new( // flattening this for now because CsvExec does not support partitioning yet files.into_iter().flatten().map(|f| f.path).collect(), statistics, @@ -72,7 +72,7 @@ impl FileFormat for AvroFormat { projection.clone(), batch_size, limit, - )?; + ); Ok(Arc::new(exec)) } } diff --git a/datafusion/src/datasource/format/csv.rs b/datafusion/src/datasource/format/csv.rs index 4b3e229f3309..1d9cd6c97765 100644 --- a/datafusion/src/datasource/format/csv.rs +++ b/datafusion/src/datasource/format/csv.rs @@ -86,7 +86,7 @@ impl FileFormat for CsvFormat { _filters: &[Expr], limit: Option, ) -> Result> { - let exec = CsvExec::try_new( + let exec = CsvExec::new( // flattening this for now because CsvExec does not support partitioning yet files.into_iter().flatten().map(|f| f.path).collect(), statistics, @@ -96,7 +96,7 @@ impl FileFormat for CsvFormat { projection.clone(), batch_size, limit, - )?; + ); Ok(Arc::new(exec)) } } diff --git a/datafusion/src/datasource/format/json.rs b/datafusion/src/datasource/format/json.rs index c2b9628423d8..a82aebd2d1eb 100644 --- a/datafusion/src/datasource/format/json.rs +++ b/datafusion/src/datasource/format/json.rs @@ -82,7 +82,7 @@ impl FileFormat for JsonFormat { _filters: &[Expr], limit: Option, ) -> Result> { - let exec = NdJsonExec::try_new( + let exec = NdJsonExec::new( // flattening this for now because NdJsonExec does not support partitioning yet files.into_iter().flatten().map(|f| f.path).collect(), statistics, @@ -90,7 +90,7 @@ impl FileFormat for JsonFormat { projection.clone(), batch_size, limit, - )?; + ); Ok(Arc::new(exec)) } } diff --git a/datafusion/src/datasource/format/parquet.rs b/datafusion/src/datasource/format/parquet.rs index 8376f9936f27..c61919587b20 100644 --- a/datafusion/src/datasource/format/parquet.rs +++ b/datafusion/src/datasource/format/parquet.rs @@ -87,7 +87,7 @@ impl FileFormat for ParquetFormat { None }; - Ok(Arc::new(ParquetExec::try_new( + Ok(Arc::new(ParquetExec::new( files, statistics, schema, @@ -97,7 +97,7 @@ impl FileFormat for ParquetFormat { .map(|l| std::cmp::min(l, batch_size)) .unwrap_or(batch_size), limit, - )?)) + ))) } } diff --git a/datafusion/src/physical_plan/format/avro.rs b/datafusion/src/physical_plan/format/avro.rs index 4163500bd393..471e95fa8a20 100644 --- a/datafusion/src/physical_plan/format/avro.rs +++ b/datafusion/src/physical_plan/format/avro.rs @@ -53,14 +53,14 @@ pub struct AvroExec { impl AvroExec { /// Create a new JSON reader execution plan provided file list and schema /// TODO: support partitiond file list (Vec>) - pub fn try_new( + pub fn new( files: Vec, statistics: Statistics, schema: SchemaRef, projection: Option>, batch_size: usize, limit: Option, - ) -> Result { + ) -> Self { let projected_schema = match &projection { None => Arc::clone(&schema), Some(p) => Arc::new(Schema::new( @@ -68,7 +68,7 @@ impl AvroExec { )), }; - Ok(Self { + Self { files, statistics, schema, @@ -76,7 +76,7 @@ impl AvroExec { projected_schema, batch_size, limit, - }) + } } } @@ -239,7 +239,7 @@ mod tests { let testdata = crate::test_util::arrow_test_data(); let filename = format!("{}/avro/alltypes_plain.avro", testdata); - let avro_exec = AvroExec::try_new( + let avro_exec = AvroExec::new( vec![filename.clone()], Statistics::default(), AvroFormat {} @@ -248,7 +248,7 @@ mod tests { Some(vec![0, 1, 2]), 1024, None, - )?; + ); assert_eq!(avro_exec.output_partitioning().partition_count(), 1); let mut results = avro_exec.execute(0).await?; diff --git a/datafusion/src/physical_plan/format/csv.rs b/datafusion/src/physical_plan/format/csv.rs index cb4013af6813..df3d74761564 100644 --- a/datafusion/src/physical_plan/format/csv.rs +++ b/datafusion/src/physical_plan/format/csv.rs @@ -63,7 +63,7 @@ impl CsvExec { /// Create a new CSV reader execution plan provided file list and schema /// TODO: support partitiond file list (Vec>) #[allow(clippy::too_many_arguments)] - pub fn try_new( + pub fn new( files: Vec, statistics: Statistics, schema: SchemaRef, @@ -72,7 +72,7 @@ impl CsvExec { projection: Option>, batch_size: usize, limit: Option, - ) -> Result { + ) -> Self { let projected_schema = match &projection { None => Arc::clone(&schema), Some(p) => Arc::new(Schema::new( @@ -80,7 +80,7 @@ impl CsvExec { )), }; - Ok(Self { + Self { files, schema, statistics, @@ -90,7 +90,7 @@ impl CsvExec { projected_schema, batch_size, limit, - }) + } } } @@ -246,7 +246,7 @@ mod tests { let testdata = crate::test_util::arrow_test_data(); let filename = "aggregate_test_100.csv"; let path = format!("{}/csv/{}", testdata, filename); - let csv = CsvExec::try_new( + let csv = CsvExec::new( vec![path], Statistics::default(), schema, @@ -255,7 +255,7 @@ mod tests { Some(vec![0, 2, 4]), 1024, None, - )?; + ); assert_eq!(13, csv.schema.fields().len()); assert_eq!(3, csv.projected_schema.fields().len()); assert_eq!(3, csv.schema().fields().len()); @@ -276,7 +276,7 @@ mod tests { let testdata = crate::test_util::arrow_test_data(); let filename = "aggregate_test_100.csv"; let path = format!("{}/csv/{}", testdata, filename); - let csv = CsvExec::try_new( + let csv = CsvExec::new( vec![path], Statistics::default(), schema, @@ -285,7 +285,7 @@ mod tests { None, 1024, None, - )?; + ); assert_eq!(13, csv.schema.fields().len()); assert_eq!(13, csv.projected_schema.fields().len()); assert_eq!(13, csv.schema().fields().len()); diff --git a/datafusion/src/physical_plan/format/json.rs b/datafusion/src/physical_plan/format/json.rs index 5ab94b142416..b187e64211ac 100644 --- a/datafusion/src/physical_plan/format/json.rs +++ b/datafusion/src/physical_plan/format/json.rs @@ -54,14 +54,14 @@ pub struct NdJsonExec { impl NdJsonExec { /// Create a new JSON reader execution plan provided file list and schema /// TODO: support partitiond file list (Vec>) - pub fn try_new( + pub fn new( files: Vec, statistics: Statistics, schema: SchemaRef, projection: Option>, batch_size: usize, limit: Option, - ) -> Result { + ) -> Self { let projected_schema = match &projection { None => Arc::clone(&schema), Some(p) => Arc::new(Schema::new( @@ -69,7 +69,7 @@ impl NdJsonExec { )), }; - Ok(Self { + Self { files, statistics, schema, @@ -77,7 +77,7 @@ impl NdJsonExec { projected_schema, batch_size, limit, - }) + } } } @@ -239,14 +239,14 @@ mod tests { async fn nd_json_exec_file_without_projection() -> Result<()> { use arrow::datatypes::DataType; let path = format!("{}/1.json", TEST_DATA_BASE); - let exec = NdJsonExec::try_new( + let exec = NdJsonExec::new( vec![path.clone()], Default::default(), infer_schema(path).await?, None, 1024, Some(3), - )?; + ); // TODO: this is not where schema inference should be tested @@ -291,14 +291,14 @@ mod tests { #[tokio::test] async fn nd_json_exec_file_projection() -> Result<()> { let path = format!("{}/1.json", TEST_DATA_BASE); - let exec = NdJsonExec::try_new( + let exec = NdJsonExec::new( vec![path.clone()], Default::default(), infer_schema(path).await?, Some(vec![0, 2]), 1024, None, - )?; + ); let inferred_schema = exec.schema(); assert_eq!(inferred_schema.fields().len(), 2); diff --git a/datafusion/src/physical_plan/format/parquet.rs b/datafusion/src/physical_plan/format/parquet.rs index fb622b5217ed..0b216631e668 100644 --- a/datafusion/src/physical_plan/format/parquet.rs +++ b/datafusion/src/physical_plan/format/parquet.rs @@ -109,7 +109,7 @@ struct ParquetFileMetrics { impl ParquetExec { /// Create a new Parquet reader execution plan provided file list and schema - pub fn try_new( + pub fn new( files: Vec>, statistics: Statistics, schema: SchemaRef, @@ -117,7 +117,7 @@ impl ParquetExec { predicate: Option, batch_size: usize, limit: Option, - ) -> Result { + ) -> Self { debug!("Creating ParquetExec, desc: {:?}, projection {:?}, predicate: {:?}, limit: {:?}", files, projection, predicate, limit); @@ -147,35 +147,31 @@ impl ParquetExec { } }); - Ok(Self::new( + let projection = match projection { + Some(p) => p, + None => (0..schema.fields().len()).collect(), + }; + + let (projected_schema, projected_statistics) = + Self::project(&projection, schema, statistics); + + Self { partitions, - schema, + schema: projected_schema, projection, - statistics, metrics, predicate_builder, batch_size, + statistics: projected_statistics, limit, - )) + } } - /// Create a new Parquet reader execution plan with provided partitions and schema - #[allow(clippy::too_many_arguments)] - fn new( - partitions: Vec, + fn project( + projection: &[usize], schema: SchemaRef, - projection: Option>, statistics: Statistics, - metrics: ExecutionPlanMetricsSet, - predicate_builder: Option, - batch_size: usize, - limit: Option, - ) -> Self { - let projection = match projection { - Some(p) => p, - None => (0..schema.fields().len()).collect(), - }; - + ) -> (SchemaRef, Statistics) { let projected_schema = Schema::new( projection .iter() @@ -185,7 +181,7 @@ impl ParquetExec { let new_column_statistics = statistics.column_statistics.map(|stats| { let mut projected_stats = Vec::with_capacity(projection.len()); - for proj in &projection { + for proj in projection { projected_stats.push(stats[*proj].clone()); } projected_stats @@ -198,31 +194,7 @@ impl ParquetExec { is_exact: statistics.is_exact, }; - Self { - partitions, - schema: Arc::new(projected_schema), - projection, - metrics, - predicate_builder, - batch_size, - statistics, - limit, - } - } - - /// Parquet partitions to read - pub fn partitions(&self) -> &[ParquetPartition] { - &self.partitions - } - - /// Projection for which columns to load - pub fn projection(&self) -> &[usize] { - &self.projection - } - - /// Batch size - pub fn batch_size(&self) -> usize { - self.batch_size + (Arc::new(projected_schema), statistics) } } @@ -569,7 +541,7 @@ mod tests { async fn test() -> Result<()> { let testdata = crate::test_util::parquet_test_data(); let filename = format!("{}/alltypes_plain.parquet", testdata); - let parquet_exec = ParquetExec::try_new( + let parquet_exec = ParquetExec::new( vec![vec![PartitionedFile { path: filename.clone(), statistics: Statistics::default(), @@ -584,7 +556,7 @@ mod tests { None, 1024, None, - )?; + ); assert_eq!(parquet_exec.output_partitioning().partition_count(), 1); let mut results = parquet_exec.execute(0).await?; From 7e0e24f60906358a84168bfc99b4bd700b7cc9ec Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 12 Oct 2021 19:24:15 +0200 Subject: [PATCH 13/39] [fix] remove try_ from ListingTable new --- datafusion/src/datasource/listing.rs | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/datafusion/src/datasource/listing.rs b/datafusion/src/datasource/listing.rs index a4c5a2b1bb66..9ee39774cae4 100644 --- a/datafusion/src/datasource/listing.rs +++ b/datafusion/src/datasource/listing.rs @@ -87,18 +87,18 @@ pub struct ListingTable { impl ListingTable { /// Create new table that lists the FS to get the files to scan. - pub fn try_new( + pub fn new( path: impl Into, // the schema must be resolved before creating the table schema: SchemaRef, options: ListingOptions, - ) -> Result { + ) -> Self { let path: String = path.into(); - Ok(Self { + Self { path, schema, options, - }) + } } } @@ -284,7 +284,7 @@ mod tests { }; // here we resolve the schema locally let schema = opt.infer_schema(&filename).await.expect("Infer schema"); - let table = ListingTable::try_new(&filename, schema, opt)?; + let table = ListingTable::new(&filename, schema, opt); Ok(Arc::new(table)) } } From abeccce0ff5190429fd6122aa9a76242eb9be40e Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 12 Oct 2021 19:24:15 +0200 Subject: [PATCH 14/39] [refacto] renamed format module to file_format also removed statistics from the PartitionedFile abstraction --- .../{format => file_format}/avro.rs | 13 ++- .../datasource/{format => file_format}/csv.rs | 11 +-- .../{format => file_format}/json.rs | 7 +- .../datasource/{format => file_format}/mod.rs | 88 +++++++++---------- .../{format => file_format}/parquet.rs | 11 +-- datafusion/src/datasource/listing.rs | 65 ++++++++------ datafusion/src/datasource/mod.rs | 2 +- .../{format => file_format}/avro.rs | 2 +- .../{format => file_format}/csv.rs | 0 .../{format => file_format}/json.rs | 2 +- .../{format => file_format}/mod.rs | 0 .../{format => file_format}/parquet.rs | 5 +- datafusion/src/physical_plan/mod.rs | 2 +- 13 files changed, 99 insertions(+), 109 deletions(-) rename datafusion/src/datasource/{format => file_format}/avro.rs (97%) rename datafusion/src/datasource/{format => file_format}/csv.rs (95%) rename datafusion/src/datasource/{format => file_format}/json.rs (96%) rename datafusion/src/datasource/{format => file_format}/mod.rs (74%) rename datafusion/src/datasource/{format => file_format}/parquet.rs (98%) rename datafusion/src/physical_plan/{format => file_format}/avro.rs (99%) rename datafusion/src/physical_plan/{format => file_format}/csv.rs (100%) rename datafusion/src/physical_plan/{format => file_format}/json.rs (99%) rename datafusion/src/physical_plan/{format => file_format}/mod.rs (100%) rename datafusion/src/physical_plan/{format => file_format}/parquet.rs (99%) diff --git a/datafusion/src/datasource/format/avro.rs b/datafusion/src/datasource/file_format/avro.rs similarity index 97% rename from datafusion/src/datasource/format/avro.rs rename to datafusion/src/datasource/file_format/avro.rs index 52f52d1d6d8c..1e8bba7c68d0 100644 --- a/datafusion/src/datasource/format/avro.rs +++ b/datafusion/src/datasource/file_format/avro.rs @@ -25,12 +25,12 @@ use async_trait::async_trait; use futures::StreamExt; use std::fs::File; +use super::PartitionedFile; use super::{FileFormat, StringStream}; use crate::avro_to_arrow::read_avro_schema_from_reader; -use crate::datasource::PartitionedFile; use crate::error::Result; use crate::logical_plan::Expr; -use crate::physical_plan::format::AvroExec; +use crate::physical_plan::file_format::AvroExec; use crate::physical_plan::ExecutionPlan; use crate::physical_plan::Statistics; @@ -80,7 +80,7 @@ impl FileFormat for AvroFormat { #[cfg(test)] #[cfg(feature = "avro")] mod tests { - use crate::datasource::format::string_stream; + use crate::datasource::file_format::string_stream; use crate::physical_plan::collect; use super::*; @@ -336,10 +336,7 @@ mod tests { .infer_stats(&filename) .await .expect("Stats inference"); - let files = vec![vec![PartitionedFile { - path: filename, - statistics: stats.clone(), - }]]; + let files = vec![vec![PartitionedFile { path: filename }]]; let exec = format .create_executor(schema, files, stats, projection, batch_size, &[], None) .await?; @@ -352,7 +349,7 @@ mod tests { mod tests { use super::*; - use crate::datasource::format::string_stream; + use crate::datasource::file_format::string_stream; use crate::error::DataFusionError; #[tokio::test] diff --git a/datafusion/src/datasource/format/csv.rs b/datafusion/src/datasource/file_format/csv.rs similarity index 95% rename from datafusion/src/datasource/format/csv.rs rename to datafusion/src/datasource/file_format/csv.rs index 1d9cd6c97765..6eaabfac7390 100644 --- a/datafusion/src/datasource/format/csv.rs +++ b/datafusion/src/datasource/file_format/csv.rs @@ -25,11 +25,11 @@ use async_trait::async_trait; use futures::StreamExt; use std::fs::File; +use super::PartitionedFile; use super::{FileFormat, StringStream}; -use crate::datasource::PartitionedFile; use crate::error::Result; use crate::logical_plan::Expr; -use crate::physical_plan::format::CsvExec; +use crate::physical_plan::file_format::CsvExec; use crate::physical_plan::ExecutionPlan; use crate::physical_plan::Statistics; @@ -106,7 +106,7 @@ mod tests { use arrow::array::StringArray; use super::*; - use crate::{datasource::format::string_stream, physical_plan::collect}; + use crate::{datasource::file_format::string_stream, physical_plan::collect}; #[tokio::test] async fn read_small_batches() -> Result<()> { @@ -212,10 +212,7 @@ mod tests { .infer_stats(&filename) .await .expect("Stats inference"); - let files = vec![vec![PartitionedFile { - path: filename, - statistics: stats.clone(), - }]]; + let files = vec![vec![PartitionedFile { path: filename }]]; let exec = format .create_executor(schema, files, stats, projection, batch_size, &[], None) .await?; diff --git a/datafusion/src/datasource/format/json.rs b/datafusion/src/datasource/file_format/json.rs similarity index 96% rename from datafusion/src/datasource/format/json.rs rename to datafusion/src/datasource/file_format/json.rs index a82aebd2d1eb..5ad363e6aa91 100644 --- a/datafusion/src/datasource/format/json.rs +++ b/datafusion/src/datasource/file_format/json.rs @@ -28,11 +28,11 @@ use async_trait::async_trait; use futures::StreamExt; use std::fs::File; +use super::PartitionedFile; use super::{FileFormat, StringStream}; -use crate::datasource::PartitionedFile; use crate::error::Result; use crate::logical_plan::Expr; -use crate::physical_plan::format::NdJsonExec; +use crate::physical_plan::file_format::NdJsonExec; use crate::physical_plan::ExecutionPlan; use crate::physical_plan::Statistics; @@ -100,7 +100,7 @@ mod tests { use arrow::array::Int64Array; use super::*; - use crate::{datasource::format::string_stream, physical_plan::collect}; + use crate::{datasource::file_format::string_stream, physical_plan::collect}; #[tokio::test] async fn read_small_batches() -> Result<()> { @@ -186,7 +186,6 @@ mod tests { let stats = format.infer_stats(filename).await.expect("Stats inference"); let files = vec![vec![PartitionedFile { path: filename.to_owned(), - statistics: stats.clone(), }]]; let exec = format .create_executor(schema, files, stats, projection, batch_size, &[], None) diff --git a/datafusion/src/datasource/format/mod.rs b/datafusion/src/datasource/file_format/mod.rs similarity index 74% rename from datafusion/src/datasource/format/mod.rs rename to datafusion/src/datasource/file_format/mod.rs index a6143353e6b1..933c7c52c12c 100644 --- a/datafusion/src/datasource/format/mod.rs +++ b/datafusion/src/datasource/file_format/mod.rs @@ -25,13 +25,11 @@ pub mod parquet; use std::pin::Pin; use std::sync::Arc; -use crate::arrow::datatypes::{Schema, SchemaRef}; +use crate::arrow::datatypes::SchemaRef; +use crate::datasource::{create_max_min_accs, get_col_stats}; use crate::error::Result; use crate::logical_plan::Expr; -use crate::physical_plan::expressions::{MaxAccumulator, MinAccumulator}; -use crate::physical_plan::{Accumulator, ColumnStatistics, ExecutionPlan, Statistics}; - -use super::PartitionedFile; +use crate::physical_plan::{Accumulator, ExecutionPlan, Statistics}; use async_trait::async_trait; use futures::Stream; @@ -77,8 +75,9 @@ pub trait FileFormat: Send + Sync { /// if the optional `limit` is provided, includes only sufficient files /// 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], + all_files: &[(PartitionedFile, Statistics)], schema: SchemaRef, limit: Option, ) -> (Vec, Statistics) { @@ -92,9 +91,8 @@ pub fn get_statistics_with_limit( let mut num_rows = 0; let mut num_files = 0; let mut is_exact = true; - for file in &all_files { + for (_, file_stats) in &all_files { num_files += 1; - let file_stats = &file.statistics; 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); @@ -152,47 +150,43 @@ pub fn get_statistics_with_limit( column_statistics: column_stats, is_exact, }; - (all_files, statistics) + + let files = all_files.into_iter().map(|(f, _)| f).collect(); + + (files, statistics) } -fn create_max_min_accs( - schema: &Schema, -) -> (Vec>, Vec>) { - let max_values: Vec> = schema - .fields() - .iter() - .map(|field| MaxAccumulator::try_new(field.data_type()).ok()) - .collect::>(); - let min_values: Vec> = schema - .fields() - .iter() - .map(|field| MinAccumulator::try_new(field.data_type()).ok()) - .collect::>(); - (max_values, min_values) +#[derive(Debug, Clone)] +/// A single file that should be read, along with its schema, statistics +/// and partition column values that need to be appended to each row. +/// 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, + // 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 } -fn get_col_stats( - schema: &Schema, - null_counts: Vec, - max_values: &mut Vec>, - min_values: &mut Vec>, -) -> Vec { - (0..schema.fields().len()) - .map(|i| { - let max_value = match &max_values[i] { - Some(max_value) => max_value.evaluate().ok(), - None => None, - }; - let min_value = match &min_values[i] { - Some(min_value) => min_value.evaluate().ok(), - None => None, - }; - ColumnStatistics { - null_count: Some(null_counts[i] as usize), - max_value, - min_value, - distinct_count: None, - } - }) - .collect() +impl std::fmt::Display for PartitionedFile { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + write!(f, "{}", self.path) + } +} + +#[derive(Debug, Clone)] +/// A collection of files that should be read in a single task +/// TODO move back to crate::datasource::mod.rs once legacy cleaned up +pub struct FilePartition { + /// The index of the partition among all partitions + pub index: usize, + /// The contained files of the partition + pub files: Vec, +} + +impl std::fmt::Display for FilePartition { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + let files: Vec = self.files.iter().map(|f| f.to_string()).collect(); + write!(f, "{}", files.join(", ")) + } } diff --git a/datafusion/src/datasource/format/parquet.rs b/datafusion/src/datasource/file_format/parquet.rs similarity index 98% rename from datafusion/src/datasource/format/parquet.rs rename to datafusion/src/datasource/file_format/parquet.rs index c61919587b20..e746d81a3172 100644 --- a/datafusion/src/datasource/format/parquet.rs +++ b/datafusion/src/datasource/file_format/parquet.rs @@ -30,15 +30,15 @@ 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 crate::arrow::datatypes::{DataType, Field}; -use crate::datasource::PartitionedFile; 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::format::ParquetExec; +use crate::physical_plan::file_format::ParquetExec; use crate::physical_plan::ExecutionPlan; use crate::physical_plan::{Accumulator, Statistics}; use crate::scalar::ScalarValue; @@ -284,7 +284,7 @@ fn fetch_metadata(path: &str) -> Result<(Schema, Statistics)> { #[cfg(test)] mod tests { - use crate::datasource::format::string_stream; + use crate::datasource::file_format::string_stream; use crate::physical_plan::collect; use super::*; @@ -532,10 +532,7 @@ mod tests { .infer_stats(&filename.clone()) .await .expect("Stats inference"); - let files = vec![vec![PartitionedFile { - path: filename, - statistics: stats.clone(), - }]]; + let files = vec![vec![PartitionedFile { path: filename }]]; let exec = format .create_executor(schema, files, stats, projection, batch_size, &[], None) .await?; diff --git a/datafusion/src/datasource/listing.rs b/datafusion/src/datasource/listing.rs index 9ee39774cae4..d240f650a573 100644 --- a/datafusion/src/datasource/listing.rs +++ b/datafusion/src/datasource/listing.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -//! A table that uses the files system / table store listing capability +//! A table that uses the `ObjectStore` listing capability //! to get the list of files to process. use std::{any::Any, sync::Arc}; @@ -25,15 +25,14 @@ use async_trait::async_trait; use futures::{StreamExt, TryStreamExt}; use crate::{ - datasource::format::{self}, + datasource::file_format::{self, PartitionedFile}, error::Result, logical_plan::Expr, physical_plan::{common, ExecutionPlan, Statistics}, }; use super::{ - datasource::TableProviderFilterPushDown, format::FileFormat, PartitionedFile, - TableProvider, + datasource::TableProviderFilterPushDown, file_format::FileFormat, TableProvider, }; /// Options for creating a `ListingTable` @@ -61,6 +60,9 @@ pub struct ListingOptions { } impl ListingOptions { + /// Infer the schema of the files at the given path, 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). @@ -128,26 +130,24 @@ impl TableProvider for ListingTable { )?; // collect the statistics if required by the config - let mut files = file_list; - if self.options.collect_stat { - files = futures::stream::iter(files) - .then(|file| async { - let statistics = self.options.format.infer_stats(&file.path).await?; - Ok(PartitionedFile { - statistics, - path: file.path, - }) as Result - }) - .try_collect::>() - .await?; - } + let files = futures::stream::iter(file_list) + .then(|file| async { + let statistics = if self.options.collect_stat { + self.options.format.infer_stats(&file.path).await? + } else { + Statistics::default() + }; + Ok((file, statistics)) as Result<(PartitionedFile, Statistics)> + }) + .try_collect::>() + .await?; let (files, statistics) = - format::get_statistics_with_limit(&files, self.schema(), limit); + file_format::get_statistics_with_limit(&files, self.schema(), limit); let partitioned_file_lists = split_files(files, self.options.max_partitions); - // 2. create the plan + // create the execution plan self.options .format .create_executor( @@ -182,10 +182,7 @@ fn pruned_partition_list( let list_all = || { Ok(common::build_file_list(path, file_extension)? .into_iter() - .map(|f| PartitionedFile { - path: f, - statistics: Statistics::default(), - }) + .map(|f| PartitionedFile { path: f }) .collect::>()) }; if partition_names.is_empty() { @@ -216,11 +213,21 @@ mod tests { #[test] fn test_split_files() { let files = vec![ - PartitionedFile::from("a".to_string()), - PartitionedFile::from("b".to_string()), - PartitionedFile::from("c".to_string()), - PartitionedFile::from("d".to_string()), - PartitionedFile::from("e".to_string()), + PartitionedFile { + path: "a".to_owned(), + }, + PartitionedFile { + path: "b".to_owned(), + }, + PartitionedFile { + path: "c".to_owned(), + }, + PartitionedFile { + path: "d".to_owned(), + }, + PartitionedFile { + path: "e".to_owned(), + }, ]; let chunks = split_files(files.clone(), 1); @@ -275,7 +282,7 @@ mod tests { let filename = format!("{}/{}", testdata, name); let opt = ListingOptions { file_extension: "parquet".to_owned(), - format: Arc::new(format::parquet::ParquetFormat { + format: Arc::new(file_format::parquet::ParquetFormat { enable_pruning: true, }), partitions: vec![], diff --git a/datafusion/src/datasource/mod.rs b/datafusion/src/datasource/mod.rs index c60ae4c3f16e..299991a5d64f 100644 --- a/datafusion/src/datasource/mod.rs +++ b/datafusion/src/datasource/mod.rs @@ -21,7 +21,7 @@ pub mod avro; pub mod csv; pub mod datasource; pub mod empty; -pub mod format; +pub mod file_format; pub mod json; pub mod listing; pub mod memory; diff --git a/datafusion/src/physical_plan/format/avro.rs b/datafusion/src/physical_plan/file_format/avro.rs similarity index 99% rename from datafusion/src/physical_plan/format/avro.rs rename to datafusion/src/physical_plan/file_format/avro.rs index 471e95fa8a20..fd50f18bf1f0 100644 --- a/datafusion/src/physical_plan/format/avro.rs +++ b/datafusion/src/physical_plan/file_format/avro.rs @@ -235,7 +235,7 @@ mod tests { async fn test() -> Result<()> { use futures::StreamExt; - use crate::datasource::format::{avro::AvroFormat, FileFormat}; + use crate::datasource::file_format::{avro::AvroFormat, FileFormat}; let testdata = crate::test_util::arrow_test_data(); let filename = format!("{}/avro/alltypes_plain.avro", testdata); diff --git a/datafusion/src/physical_plan/format/csv.rs b/datafusion/src/physical_plan/file_format/csv.rs similarity index 100% rename from datafusion/src/physical_plan/format/csv.rs rename to datafusion/src/physical_plan/file_format/csv.rs diff --git a/datafusion/src/physical_plan/format/json.rs b/datafusion/src/physical_plan/file_format/json.rs similarity index 99% rename from datafusion/src/physical_plan/format/json.rs rename to datafusion/src/physical_plan/file_format/json.rs index b187e64211ac..69be9d2e7a9f 100644 --- a/datafusion/src/physical_plan/format/json.rs +++ b/datafusion/src/physical_plan/file_format/json.rs @@ -221,7 +221,7 @@ impl RecordBatchStream for NdJsonStream { mod tests { use futures::StreamExt; - use crate::datasource::format::{json::JsonFormat, FileFormat}; + use crate::datasource::file_format::{json::JsonFormat, FileFormat}; use super::*; diff --git a/datafusion/src/physical_plan/format/mod.rs b/datafusion/src/physical_plan/file_format/mod.rs similarity index 100% rename from datafusion/src/physical_plan/format/mod.rs rename to datafusion/src/physical_plan/file_format/mod.rs diff --git a/datafusion/src/physical_plan/format/parquet.rs b/datafusion/src/physical_plan/file_format/parquet.rs similarity index 99% rename from datafusion/src/physical_plan/format/parquet.rs rename to datafusion/src/physical_plan/file_format/parquet.rs index 0b216631e668..615e52b54474 100644 --- a/datafusion/src/physical_plan/format/parquet.rs +++ b/datafusion/src/physical_plan/file_format/parquet.rs @@ -58,7 +58,7 @@ use tokio::{ use async_trait::async_trait; -use crate::datasource::{FilePartition, PartitionedFile}; +use crate::datasource::file_format::{FilePartition, PartitionedFile}; /// Execution plan for scanning one or more Parquet partitions #[derive(Debug, Clone)] @@ -526,7 +526,7 @@ fn read_partition( #[cfg(test)] mod tests { - use crate::datasource::format::{parquet::ParquetFormat, FileFormat}; + use crate::datasource::file_format::{parquet::ParquetFormat, FileFormat}; use super::*; use arrow::datatypes::{DataType, Field}; @@ -544,7 +544,6 @@ mod tests { let parquet_exec = ParquetExec::new( vec![vec![PartitionedFile { path: filename.clone(), - statistics: Statistics::default(), }]], Statistics::default(), ParquetFormat { diff --git a/datafusion/src/physical_plan/mod.rs b/datafusion/src/physical_plan/mod.rs index 0362549874e6..373b5e3fe959 100644 --- a/datafusion/src/physical_plan/mod.rs +++ b/datafusion/src/physical_plan/mod.rs @@ -620,8 +620,8 @@ pub mod distinct_expressions; pub mod empty; pub mod explain; pub mod expressions; +pub mod file_format; pub mod filter; -pub mod format; pub mod functions; pub mod hash_aggregate; pub mod hash_join; From 8abcc977d73a9ee4eb682ab549a5a5b869ac99b0 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 12 Oct 2021 19:24:15 +0200 Subject: [PATCH 15/39] [fix] removed Ballista stubs --- ballista/rust/core/proto/ballista.proto | 26 -------------------- ballista/rust/scheduler/src/lib.rs | 11 --------- datafusion/src/datasource/file_format/mod.rs | 2 +- 3 files changed, 1 insertion(+), 38 deletions(-) diff --git a/ballista/rust/core/proto/ballista.proto b/ballista/rust/core/proto/ballista.proto index f92c7d71d1ba..9a2ec710411b 100644 --- a/ballista/rust/core/proto/ballista.proto +++ b/ballista/rust/core/proto/ballista.proto @@ -949,30 +949,6 @@ message GetFileMetadataParams { FileType file_type = 2; } -message ParquetFormat { - // fields of datasource::format::parquet::ParquetFormat -} - -message ListingConfig { - string extension = 1; - oneof format { - ParquetFormat parquet = 2; - // csv, json, ... - } -} - -message GetSchemaParams { - string path = 1; - oneof provider_config { - ListingConfig listing = 2; - // delta, iceberg... (maybe replace oneof with something more exensible) - } -} - -message GetSchemaResult { - Schema schema = 1; -} - message GetFileMetadataResult { Schema schema = 1; repeated FilePartitionMetadata partitions = 2; @@ -986,8 +962,6 @@ service SchedulerGrpc { // Executors must poll the scheduler for heartbeat and to receive tasks rpc PollWork (PollWorkParams) returns (PollWorkResult) {} - rpc GetSchema (GetSchemaParams) returns (GetSchemaResult) {} - rpc GetFileMetadata (GetFileMetadataParams) returns (GetFileMetadataResult) {} rpc ExecuteQuery (ExecuteQueryParams) returns (ExecuteQueryResult) {} diff --git a/ballista/rust/scheduler/src/lib.rs b/ballista/rust/scheduler/src/lib.rs index e57b967de8a1..47caf4c21ede 100644 --- a/ballista/rust/scheduler/src/lib.rs +++ b/ballista/rust/scheduler/src/lib.rs @@ -22,7 +22,6 @@ pub mod planner; #[cfg(feature = "sled")] mod standalone; pub mod state; -use ballista_core::serde::protobuf::{GetSchemaParams, GetSchemaResult}; #[cfg(feature = "sled")] pub use standalone::new_standalone_scheduler; @@ -269,16 +268,6 @@ impl SchedulerGrpc for SchedulerServer { } } - /// this service would replace get_file_metadata - async fn get_schema( - &self, - _request: Request, - ) -> std::result::Result, tonic::Status> { - // if GetSchemaParams contains a provider config of type ListingConfig - // use ListingOptions.infer_schema() - todo!() - } - async fn get_file_metadata( &self, request: Request, diff --git a/datafusion/src/datasource/file_format/mod.rs b/datafusion/src/datasource/file_format/mod.rs index 933c7c52c12c..d831919cc9ba 100644 --- a/datafusion/src/datasource/file_format/mod.rs +++ b/datafusion/src/datasource/file_format/mod.rs @@ -37,7 +37,7 @@ use futures::Stream; /// A stream of String that can be used accross await calls pub type StringStream = Pin + Send + Sync>>; -/// Convert a vec into a `StringStream` +/// Convert a vector into a stream pub fn string_stream(strings: Vec) -> StringStream { Box::pin(futures::stream::iter(strings)) } From c1530a9f0cc330734561c9f7bb755d30b45eea49 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 12 Oct 2021 19:24:15 +0200 Subject: [PATCH 16/39] [fix] rename create_executor --- datafusion/src/datasource/file_format/avro.rs | 4 ++-- datafusion/src/datasource/file_format/csv.rs | 4 ++-- datafusion/src/datasource/file_format/json.rs | 4 ++-- datafusion/src/datasource/file_format/mod.rs | 2 +- datafusion/src/datasource/file_format/parquet.rs | 4 ++-- datafusion/src/datasource/listing.rs | 2 +- 6 files changed, 10 insertions(+), 10 deletions(-) diff --git a/datafusion/src/datasource/file_format/avro.rs b/datafusion/src/datasource/file_format/avro.rs index 1e8bba7c68d0..bb5083d547b7 100644 --- a/datafusion/src/datasource/file_format/avro.rs +++ b/datafusion/src/datasource/file_format/avro.rs @@ -54,7 +54,7 @@ impl FileFormat for AvroFormat { Ok(Statistics::default()) } - async fn create_executor( + async fn create_physical_plan( &self, schema: SchemaRef, files: Vec>, @@ -338,7 +338,7 @@ mod tests { .expect("Stats inference"); let files = vec![vec![PartitionedFile { path: filename }]]; let exec = format - .create_executor(schema, files, stats, projection, batch_size, &[], None) + .create_physical_plan(schema, files, stats, projection, batch_size, &[], None) .await?; Ok(exec) } diff --git a/datafusion/src/datasource/file_format/csv.rs b/datafusion/src/datasource/file_format/csv.rs index 6eaabfac7390..a5b555053ef7 100644 --- a/datafusion/src/datasource/file_format/csv.rs +++ b/datafusion/src/datasource/file_format/csv.rs @@ -76,7 +76,7 @@ impl FileFormat for CsvFormat { Ok(Statistics::default()) } - async fn create_executor( + async fn create_physical_plan( &self, schema: SchemaRef, files: Vec>, @@ -214,7 +214,7 @@ mod tests { .expect("Stats inference"); let files = vec![vec![PartitionedFile { path: filename }]]; let exec = format - .create_executor(schema, files, stats, projection, batch_size, &[], None) + .create_physical_plan(schema, files, stats, projection, batch_size, &[], None) .await?; Ok(exec) } diff --git a/datafusion/src/datasource/file_format/json.rs b/datafusion/src/datasource/file_format/json.rs index 5ad363e6aa91..7357644af1fa 100644 --- a/datafusion/src/datasource/file_format/json.rs +++ b/datafusion/src/datasource/file_format/json.rs @@ -72,7 +72,7 @@ impl FileFormat for JsonFormat { Ok(Statistics::default()) } - async fn create_executor( + async fn create_physical_plan( &self, schema: SchemaRef, files: Vec>, @@ -188,7 +188,7 @@ mod tests { path: filename.to_owned(), }]]; let exec = format - .create_executor(schema, files, stats, projection, batch_size, &[], None) + .create_physical_plan(schema, files, stats, projection, batch_size, &[], None) .await?; Ok(exec) } diff --git a/datafusion/src/datasource/file_format/mod.rs b/datafusion/src/datasource/file_format/mod.rs index d831919cc9ba..e08ebd615c80 100644 --- a/datafusion/src/datasource/file_format/mod.rs +++ b/datafusion/src/datasource/file_format/mod.rs @@ -59,7 +59,7 @@ pub trait FileFormat: Send + Sync { /// 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_executor( + async fn create_physical_plan( &self, schema: SchemaRef, files: Vec>, diff --git a/datafusion/src/datasource/file_format/parquet.rs b/datafusion/src/datasource/file_format/parquet.rs index e746d81a3172..39aa14de0b9f 100644 --- a/datafusion/src/datasource/file_format/parquet.rs +++ b/datafusion/src/datasource/file_format/parquet.rs @@ -68,7 +68,7 @@ impl FileFormat for ParquetFormat { Ok(stats) } - async fn create_executor( + async fn create_physical_plan( &self, schema: SchemaRef, files: Vec>, @@ -534,7 +534,7 @@ mod tests { .expect("Stats inference"); let files = vec![vec![PartitionedFile { path: filename }]]; let exec = format - .create_executor(schema, files, stats, projection, batch_size, &[], None) + .create_physical_plan(schema, files, stats, projection, batch_size, &[], None) .await?; Ok(exec) } diff --git a/datafusion/src/datasource/listing.rs b/datafusion/src/datasource/listing.rs index d240f650a573..cac9b36b3d51 100644 --- a/datafusion/src/datasource/listing.rs +++ b/datafusion/src/datasource/listing.rs @@ -150,7 +150,7 @@ impl TableProvider for ListingTable { // create the execution plan self.options .format - .create_executor( + .create_physical_plan( self.schema(), partitioned_file_lists, statistics, From a763512ba16b3f57746c14bebd6e9c8a58049e8b Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 12 Oct 2021 19:24:15 +0200 Subject: [PATCH 17/39] [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 531b1092e1d6..f30fbdcc0cec 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 bb5083d547b7..c6f7b24f0e7d 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 a5b555053ef7..288c9c6faa6f 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 7357644af1fa..02f5fcd5a955 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 e08ebd615c80..8ce30a5a9a15 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 39aa14de0b9f..26b49d76cc97 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 cac9b36b3d51..3798aaff4686 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 2b27f6c8f993..18e74e2e1161 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 fd25fd43a2e7..08bbad020eb5 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 fd50f18bf1f0..5259780e16ea 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 df3d74761564..9989c3dd8bb1 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 69be9d2e7a9f..923cd1bb5ecb 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 615e52b54474..f745365c835a 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 e71229d64dfe7c671a6e10793a204766e031bcc4 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 12 Oct 2021 19:24:15 +0200 Subject: [PATCH 18/39] [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 c6f7b24f0e7d..dc3a253dbe46 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 3798aaff4686..5c7b46929026 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 9989c3dd8bb1..ca8f922b1e9f 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 f745365c835a..0ae1b073a22c 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 f72326227a8c59c9af18f10c8383f83cfcc464f3 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 12 Oct 2021 19:24:15 +0200 Subject: [PATCH 19/39] [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 dc3a253dbe46..cd2f94cea66b 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 288c9c6faa6f..d8ce6282bdf0 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 02f5fcd5a955..5512497e3289 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 26b49d76cc97..99a8a890914f 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 0ae1b073a22c..433182abb7bf 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 6a7c36a7171036717e36fd300f054e04eaaf6811 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 12 Oct 2021 19:24:15 +0200 Subject: [PATCH 20/39] [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 18e74e2e1161..a99616f08ec5 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 ccc7938b0a271a68ac2d2599eb9c9efd0e612512 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 12 Oct 2021 19:24:15 +0200 Subject: [PATCH 21/39] [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 8ce30a5a9a15..b6e5f8a3f277 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 5c7b46929026..bd5ba6804dc7 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 a99616f08ec5..a68de29e003f 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 08bbad020eb5..ab4735ccacf5 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 4dbe20184854d33a4aef51a8118fc4202d5df97a Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 12 Oct 2021 19:24:15 +0200 Subject: [PATCH 22/39] [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 cd2f94cea66b..0e6008799271 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 d8ce6282bdf0..9f8b27cb9114 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 5512497e3289..20441b2d62c2 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 ab4735ccacf5..a63f55e447d4 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 5259780e16ea..e20ba3afe7a8 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 ca8f922b1e9f..d7392579774c 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 923cd1bb5ecb..1d02f5fca3b2 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 687993890199bfda491dc20eaec6fa05bfb18897 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 12 Oct 2021 19:24:15 +0200 Subject: [PATCH 23/39] [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 a68de29e003f..8fe46fad37ae 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 a63f55e447d4..b03c795024b2 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 1947a874cbc6e4d394f49007d153c011cbdb4a2d Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 12 Oct 2021 19:24:15 +0200 Subject: [PATCH 24/39] [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 0e6008799271..ac83d63296ae 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 9f8b27cb9114..302afb4c0d9c 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 20441b2d62c2..ffa6216cf739 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 99a8a890914f..6e69651fc407 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 8fe46fad37ae..655e20627415 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 b03c795024b2..3474f691f481 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 e20ba3afe7a8..65b5b6e4b4e9 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 d7392579774c..5aa7e62a112a 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 1d02f5fca3b2..7a81a75fadc5 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 aa7cbad19c4c3adf60593e4d0d14b7a03012862f Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 12 Oct 2021 19:24:15 +0200 Subject: [PATCH 25/39] [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 b6e5f8a3f277..a23816a7f011 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 d6a718b335eb0b0ea769c75592e1735a02c82fc6 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 12 Oct 2021 19:24:15 +0200 Subject: [PATCH 26/39] [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 bd5ba6804dc7..a577299e9de6 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 1be110ca586e203418764be42c59ddff9bf5ed67 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 12 Oct 2021 19:24:15 +0200 Subject: [PATCH 27/39] [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 a577299e9de6..303c42503aa0 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 655e20627415..d9006e2b5a76 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 3474f691f481..20e43021a8fc 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 cbfbdc17ec951ed12b7cca8c494f85c33bfcbb62 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 12 Oct 2021 19:24:15 +0200 Subject: [PATCH 28/39] [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 5aa7e62a112a..9df7a3bef3c7 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 3a4f6d8543aafece90fb0177b9c7d3487fcfe207 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 12 Oct 2021 19:24:15 +0200 Subject: [PATCH 29/39] [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 ac83d63296ae..8f46affe8656 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 302afb4c0d9c..ea47223cc60a 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 ffa6216cf739..f7ed8517ae63 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 a23816a7f011..a876cac5e4e7 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 6e69651fc407..da132c5d7668 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 303c42503aa0..ff28d3b21470 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 d9006e2b5a76..2a331d8e6f15 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 20e43021a8fc..7a6f024595ab 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 65b5b6e4b4e9..d079748ec85f 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 9df7a3bef3c7..e662f7cecbec 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 7a81a75fadc5..4e62c5e29840 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 433182abb7bf..1be3d8b3284a 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 d20eff3d58e512f1e05ab1731b4bce9d6b8702ee Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 12 Oct 2021 19:24:15 +0200 Subject: [PATCH 30/39] [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 7a6f024595ab..6b932f1c4ee3 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 7a4a25005da102ba804cfae67707186cb79ee387 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 12 Oct 2021 19:24:15 +0200 Subject: [PATCH 31/39] [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 8f46affe8656..cfe15090e23f 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 ea47223cc60a..1abdca70ae2e 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 f7ed8517ae63..4e3ef37138df 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 a876cac5e4e7..d1ebc443e122 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 da132c5d7668..a49095f109ba 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 ff28d3b21470..5e9da121ac73 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 2a331d8e6f15..c18af1cd8f44 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 6b932f1c4ee3..eb4ede2e4618 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 d079748ec85f..23d912803250 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 e662f7cecbec..df6022cc44aa 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 4e62c5e29840..090fd3f59ef3 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 1be3d8b3284a..3d73d0550fa0 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 cac1c877224a1344fcd9e6994b6327bf775c3ec8 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 12 Oct 2021 19:24:15 +0200 Subject: [PATCH 32/39] [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 cfe15090e23f..1ca538867cca 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 1abdca70ae2e..d619227e4306 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 4e3ef37138df..fd7f6580ab8a 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 d1ebc443e122..b315c99b7d14 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 a49095f109ba..9e5d623b7bb1 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 5e9da121ac73..b1370bec3efb 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 ab3fcdd9972767bb63d2c78d4081271440fdc1d2 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 12 Oct 2021 19:24:15 +0200 Subject: [PATCH 33/39] [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 b1370bec3efb..f4b8f7755166 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 eb4ede2e4618..b16684a9db0a 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)) } } } From 8f4e24292fed9e3dc619da1ef42a0273edcb726b Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 12 Oct 2021 19:24:15 +0200 Subject: [PATCH 34/39] [fix] ListingTable at store level instead of registry --- datafusion/src/datasource/listing.rs | 56 +++++++++++++--------------- 1 file changed, 26 insertions(+), 30 deletions(-) diff --git a/datafusion/src/datasource/listing.rs b/datafusion/src/datasource/listing.rs index f4b8f7755166..1e048e12d98d 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, PhysicalPlanConfig}, - object_store::{ObjectStore, ObjectStoreRegistry}, + object_store::ObjectStore, TableProvider, }; @@ -69,12 +69,11 @@ impl ListingOptions { /// 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, - object_store_registry: Arc, - uri: &str, + pub async fn infer_schema<'a>( + &'a self, + object_store: Arc, + path: &'a str, ) -> Result { - let (object_store, path) = object_store_registry.get_by_uri(uri)?; let file_stream = object_store .list_file_with_suffix(path, &self.file_extension) .await? @@ -92,9 +91,8 @@ 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 { - // TODO pass object_store_registry to scan() instead - object_store_registry: Arc, - uri: String, + object_store: Arc, + path: String, schema: SchemaRef, options: ListingOptions, } @@ -102,17 +100,15 @@ pub struct ListingTable { impl ListingTable { /// Create new table that lists the FS to get the files to scan. pub fn new( - // TODO pass object_store_registry to scan() instead - object_store_registry: Arc, - uri: impl Into, + object_store: Arc, + path: String, // the schema must be resolved before creating the table schema: SchemaRef, options: ListingOptions, ) -> Self { - let uri: String = uri.into(); Self { - object_store_registry, - uri, + object_store, + path, schema, options, } @@ -137,15 +133,19 @@ impl TableProvider for ListingTable { limit: Option, ) -> Result> { // TODO object_store_registry should be provided as param here - 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), path, filters, limit) + .list_files_for_scan( + Arc::clone(&self.object_store), + &self.path, + filters, + limit, + ) .await?; // create the execution plan self.options .format .create_physical_plan(PhysicalPlanConfig { - object_store, + object_store: Arc::clone(&self.object_store), schema: self.schema(), files: partitioned_file_lists, statistics, @@ -205,7 +205,7 @@ impl ListingTable { if files.is_empty() { return Err(DataFusionError::Plan(format!( "No files found at {} with file extension {}", - self.uri, self.options.file_extension, + self.path, self.options.file_extension, ))); } @@ -257,8 +257,8 @@ mod tests { use crate::datasource::{ file_format::{avro::AvroFormat, parquet::ParquetFormat}, object_store::{ - FileMeta, FileMetaStream, ListEntryStream, ObjectReader, ObjectStore, - SizedFile, + local::LocalFileSystem, FileMeta, FileMetaStream, ListEntryStream, + ObjectReader, ObjectStore, SizedFile, }, }; @@ -347,13 +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(Arc::clone(&object_store_reg), &filename) + .infer_schema(Arc::new(LocalFileSystem {}), &filename) .await .expect("Infer schema"); - let table = ListingTable::new(object_store_reg, &filename, schema, opt); + let table = + ListingTable::new(Arc::new(LocalFileSystem {}), filename, schema, opt); Ok(Arc::new(table)) } @@ -362,10 +362,8 @@ mod tests { max_partitions: usize, output_partitioning: usize, ) -> Result<()> { - let registry = ObjectStoreRegistry::new(); let mock_store: Arc = Arc::new(MockObjectStore { files_in_folder }); - registry.register_store("mock".to_owned(), Arc::clone(&mock_store)); let format = AvroFormat {}; @@ -377,13 +375,11 @@ 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( - object_store_reg, - "mock://bucket/key-prefix", + Arc::clone(&mock_store), + "bucket/key-prefix".to_owned(), Arc::new(schema), opt, ); From db083725f6694c1d9233a84e41cd83d9f6b4ef03 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 12 Oct 2021 19:24:15 +0200 Subject: [PATCH 35/39] [fix] builder take self and not ref to self --- datafusion/src/datasource/file_format/avro.rs | 1 + datafusion/src/datasource/file_format/csv.rs | 6 +++--- datafusion/src/datasource/file_format/json.rs | 2 +- datafusion/src/datasource/file_format/parquet.rs | 2 +- 4 files changed, 6 insertions(+), 5 deletions(-) diff --git a/datafusion/src/datasource/file_format/avro.rs b/datafusion/src/datasource/file_format/avro.rs index 1ca538867cca..cb0f21640f51 100644 --- a/datafusion/src/datasource/file_format/avro.rs +++ b/datafusion/src/datasource/file_format/avro.rs @@ -33,6 +33,7 @@ use crate::physical_plan::ExecutionPlan; use crate::physical_plan::Statistics; /// Avro `FileFormat` implementation. +#[derive(Default)] pub struct AvroFormat; #[async_trait] diff --git a/datafusion/src/datasource/file_format/csv.rs b/datafusion/src/datasource/file_format/csv.rs index d619227e4306..c4a6462b9bd4 100644 --- a/datafusion/src/datasource/file_format/csv.rs +++ b/datafusion/src/datasource/file_format/csv.rs @@ -51,21 +51,21 @@ impl Default for CsvFormat { impl CsvFormat { /// 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 { + pub fn with_schema_infer_max_rec(mut self, max_rec: Option) -> Self { self.schema_infer_max_rec = max_rec; self } /// Set true to indicate that the first line is a header. /// - default to true - pub fn with_has_header(&mut self, has_header: bool) -> &mut Self { + pub fn with_has_header(mut self, has_header: bool) -> 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 { + pub fn with_delimiter(mut self, delimiter: u8) -> Self { self.delimiter = delimiter; self } diff --git a/datafusion/src/datasource/file_format/json.rs b/datafusion/src/datasource/file_format/json.rs index fd7f6580ab8a..2fa683437869 100644 --- a/datafusion/src/datasource/file_format/json.rs +++ b/datafusion/src/datasource/file_format/json.rs @@ -51,7 +51,7 @@ impl Default for JsonFormat { impl JsonFormat { /// 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 { + pub fn with_schema_infer_max_rec(mut self, max_rec: Option) -> Self { self.schema_infer_max_rec = max_rec; self } diff --git a/datafusion/src/datasource/file_format/parquet.rs b/datafusion/src/datasource/file_format/parquet.rs index 9e5d623b7bb1..10fe0d79ebeb 100644 --- a/datafusion/src/datasource/file_format/parquet.rs +++ b/datafusion/src/datasource/file_format/parquet.rs @@ -63,7 +63,7 @@ impl Default for ParquetFormat { impl ParquetFormat { /// Activate statistics based row group level pruning /// - defaults to true - pub fn with_enable_pruning(&mut self, enable: bool) -> &mut Self { + pub fn with_enable_pruning(mut self, enable: bool) -> Self { self.enable_pruning = enable; self } From e0a670f4ae18638afc085f8284818bdd728e35ee Mon Sep 17 00:00:00 2001 From: rdettai Date: Tue, 12 Oct 2021 09:03:42 +0200 Subject: [PATCH 36/39] Replace file format providers (#2) * [fix] replace file format providers in datafusion * [lint] clippy * [fix] replace file format providers in ballista * [fix] await in python wrapper --- README.md | 6 +- .../src/bin/ballista-dataframe.rs | 3 +- ballista-examples/src/bin/ballista-sql.rs | 11 +- ballista/rust/client/README.md | 4 +- ballista/rust/client/src/context.rs | 59 +- ballista/rust/core/Cargo.toml | 1 + ballista/rust/core/proto/ballista.proto | 96 +- .../core/src/serde/logical_plan/from_proto.rs | 185 ++-- .../rust/core/src/serde/logical_plan/mod.rs | 62 +- .../core/src/serde/logical_plan/to_proto.rs | 139 +-- ballista/rust/core/src/serde/mod.rs | 14 + .../src/serde/physical_plan/from_proto.rs | 153 ++- .../core/src/serde/physical_plan/to_proto.rs | 136 ++- ballista/rust/core/src/utils.rs | 3 +- ballista/rust/scheduler/src/lib.rs | 70 +- ballista/rust/scheduler/src/planner.rs | 30 +- ballista/rust/scheduler/src/test_utils.rs | 6 +- benchmarks/src/bin/nyctaxi.rs | 6 +- benchmarks/src/bin/tpch.rs | 119 ++- datafusion-cli/src/main.rs | 4 +- datafusion-examples/examples/avro_sql.rs | 12 +- datafusion-examples/examples/csv_sql.rs | 11 +- datafusion-examples/examples/dataframe.rs | 3 +- datafusion-examples/examples/flight_server.rs | 18 +- datafusion-examples/examples/parquet_sql.rs | 11 +- datafusion/benches/aggregate_query_sql.rs | 2 +- datafusion/benches/filter_query_sql.rs | 2 +- datafusion/benches/math_query_sql.rs | 2 +- datafusion/benches/sort_limit_query_sql.rs | 19 +- datafusion/benches/window_query_sql.rs | 2 +- datafusion/src/dataframe.rs | 78 +- datafusion/src/datasource/avro.rs | 426 --------- datafusion/src/datasource/csv.rs | 245 ----- datafusion/src/datasource/file_format/avro.rs | 11 +- datafusion/src/datasource/file_format/csv.rs | 21 +- datafusion/src/datasource/file_format/json.rs | 11 +- datafusion/src/datasource/file_format/mod.rs | 140 +-- .../src/datasource/file_format/parquet.rs | 15 +- datafusion/src/datasource/json.rs | 184 ---- datafusion/src/datasource/listing.rs | 51 +- datafusion/src/datasource/mod.rs | 236 ++--- datafusion/src/datasource/parquet.rs | 677 ------------- datafusion/src/execution/context.rs | 326 +++++-- datafusion/src/execution/dataframe_impl.rs | 69 +- datafusion/src/execution/mod.rs | 1 + datafusion/src/execution/options.rs | 173 ++++ datafusion/src/lib.rs | 6 +- datafusion/src/logical_plan/builder.rs | 146 ++- .../src/physical_optimizer/repartition.rs | 26 +- datafusion/src/physical_plan/avro.rs | 457 --------- .../src/physical_plan/coalesce_partitions.rs | 18 +- datafusion/src/physical_plan/csv.rs | 534 ----------- .../src/physical_plan/file_format/avro.rs | 30 +- .../src/physical_plan/file_format/csv.rs | 60 +- .../src/physical_plan/file_format/json.rs | 2 +- .../src/physical_plan/file_format/parquet.rs | 23 +- datafusion/src/physical_plan/filter.rs | 22 +- datafusion/src/physical_plan/json.rs | 507 ---------- datafusion/src/physical_plan/limit.rs | 17 +- datafusion/src/physical_plan/mod.rs | 9 +- datafusion/src/physical_plan/parquet.rs | 903 ------------------ datafusion/src/physical_plan/planner.rs | 105 +- datafusion/src/physical_plan/projection.rs | 22 +- datafusion/src/physical_plan/sort.rs | 25 +- .../physical_plan/sort_preserving_merge.rs | 52 +- datafusion/src/physical_plan/source.rs | 90 -- datafusion/src/physical_plan/union.rs | 39 +- datafusion/src/physical_plan/windows/mod.rs | 22 +- datafusion/src/prelude.rs | 3 +- datafusion/src/test/mod.rs | 20 +- datafusion/tests/custom_sources.rs | 1 + datafusion/tests/parquet_pruning.rs | 7 +- datafusion/tests/provider_filter_pushdown.rs | 3 +- datafusion/tests/sql.rs | 179 ++-- datafusion/tests/statistics.rs | 15 +- datafusion/tests/user_defined_plan.rs | 8 +- python/src/context.rs | 34 +- 77 files changed, 1882 insertions(+), 5356 deletions(-) delete mode 100644 datafusion/src/datasource/avro.rs delete mode 100644 datafusion/src/datasource/csv.rs delete mode 100644 datafusion/src/datasource/json.rs delete mode 100644 datafusion/src/datasource/parquet.rs create mode 100644 datafusion/src/execution/options.rs delete mode 100644 datafusion/src/physical_plan/avro.rs delete mode 100644 datafusion/src/physical_plan/csv.rs delete mode 100644 datafusion/src/physical_plan/json.rs delete mode 100644 datafusion/src/physical_plan/parquet.rs delete mode 100644 datafusion/src/physical_plan/source.rs diff --git a/README.md b/README.md index 8b129177deda..458f1974816f 100644 --- a/README.md +++ b/README.md @@ -76,10 +76,10 @@ use datafusion::arrow::record_batch::RecordBatch; async fn main() -> datafusion::error::Result<()> { // register the table let mut ctx = ExecutionContext::new(); - ctx.register_csv("example", "tests/example.csv", CsvReadOptions::new())?; + ctx.register_csv("example", "tests/example.csv", CsvReadOptions::new()).await?; // create a plan to run a SQL query - let df = ctx.sql("SELECT a, MIN(b) FROM example GROUP BY a LIMIT 100")?; + let df = ctx.sql("SELECT a, MIN(b) FROM example GROUP BY a LIMIT 100").await?; // execute and print results df.show().await?; @@ -98,7 +98,7 @@ use datafusion::arrow::record_batch::RecordBatch; async fn main() -> datafusion::error::Result<()> { // create the dataframe let mut ctx = ExecutionContext::new(); - let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new())?; + let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new()).await?; let df = df.filter(col("a").lt_eq(col("b")))? .aggregate(vec![col("a")], vec![min(col("b"))])?; diff --git a/ballista-examples/src/bin/ballista-dataframe.rs b/ballista-examples/src/bin/ballista-dataframe.rs index 434ed7bcd899..8399324ad0e2 100644 --- a/ballista-examples/src/bin/ballista-dataframe.rs +++ b/ballista-examples/src/bin/ballista-dataframe.rs @@ -33,7 +33,8 @@ async fn main() -> Result<()> { // define the query using the DataFrame trait let df = ctx - .read_parquet(filename)? + .read_parquet(filename) + .await? .select_columns(&["id", "bool_col", "timestamp_col"])? .filter(col("id").gt(lit(1)))?; diff --git a/ballista-examples/src/bin/ballista-sql.rs b/ballista-examples/src/bin/ballista-sql.rs index 4b303e3ef3d5..3e0df21a73f1 100644 --- a/ballista-examples/src/bin/ballista-sql.rs +++ b/ballista-examples/src/bin/ballista-sql.rs @@ -34,15 +34,18 @@ async fn main() -> Result<()> { "aggregate_test_100", &format!("{}/csv/aggregate_test_100.csv", testdata), CsvReadOptions::new(), - )?; + ) + .await?; // execute the query - let df = ctx.sql( - "SELECT c1, MIN(c12), MAX(c12) \ + let df = ctx + .sql( + "SELECT c1, MIN(c12), MAX(c12) \ FROM aggregate_test_100 \ WHERE c11 > 0.1 AND c11 < 0.9 \ GROUP BY c1", - )?; + ) + .await?; // print the results df.show().await?; diff --git a/ballista/rust/client/README.md b/ballista/rust/client/README.md index 6443dd25a147..08485b6f291d 100644 --- a/ballista/rust/client/README.md +++ b/ballista/rust/client/README.md @@ -104,7 +104,7 @@ async fn main() -> Result<()> { "tripdata", "/path/to/yellow_tripdata_2020-01.csv", CsvReadOptions::new(), - )?; + ).await?; // execute the query let df = ctx.sql( @@ -112,7 +112,7 @@ async fn main() -> Result<()> { FROM tripdata GROUP BY passenger_count ORDER BY passenger_count", - )?; + ).await?; // collect the results and print them to stdout let results = df.collect().await?; diff --git a/ballista/rust/client/src/context.rs b/ballista/rust/client/src/context.rs index c8df5099c2be..615e6c981281 100644 --- a/ballista/rust/client/src/context.rs +++ b/ballista/rust/client/src/context.rs @@ -31,8 +31,7 @@ use datafusion::datasource::TableProvider; use datafusion::error::{DataFusionError, Result}; use datafusion::execution::dataframe_impl::DataFrameImpl; use datafusion::logical_plan::LogicalPlan; -use datafusion::physical_plan::avro::AvroReadOptions; -use datafusion::physical_plan::csv::CsvReadOptions; +use datafusion::prelude::{AvroReadOptions, CsvReadOptions}; use datafusion::sql::parser::FileType; struct BallistaContextState { @@ -128,11 +127,11 @@ impl BallistaContext { } /// Create a DataFrame representing an Avro table scan - - pub fn read_avro( + /// TODO fetch schema from scheduler instead of resolving locally + pub async fn read_avro( &self, path: &str, - options: AvroReadOptions, + options: AvroReadOptions<'_>, ) -> Result> { // convert to absolute path because the executor likely has a different working directory let path = PathBuf::from(path); @@ -147,13 +146,13 @@ impl BallistaContext { guard.config(), ) }; - let df = ctx.read_avro(path.to_str().unwrap(), options)?; + let df = ctx.read_avro(path.to_str().unwrap(), options).await?; Ok(df) } /// Create a DataFrame representing a Parquet table scan - - pub fn read_parquet(&self, path: &str) -> Result> { + /// TODO fetch schema from scheduler instead of resolving locally + pub async fn read_parquet(&self, path: &str) -> Result> { // convert to absolute path because the executor likely has a different working directory let path = PathBuf::from(path); let path = fs::canonicalize(&path)?; @@ -167,16 +166,16 @@ impl BallistaContext { guard.config(), ) }; - let df = ctx.read_parquet(path.to_str().unwrap())?; + let df = ctx.read_parquet(path.to_str().unwrap()).await?; Ok(df) } /// Create a DataFrame representing a CSV table scan - - pub fn read_csv( + /// TODO fetch schema from scheduler instead of resolving locally + pub async fn read_csv( &self, path: &str, - options: CsvReadOptions, + options: CsvReadOptions<'_>, ) -> Result> { // convert to absolute path because the executor likely has a different working directory let path = PathBuf::from(path); @@ -191,7 +190,7 @@ impl BallistaContext { guard.config(), ) }; - let df = ctx.read_csv(path.to_str().unwrap(), options)?; + let df = ctx.read_csv(path.to_str().unwrap(), options).await?; Ok(df) } @@ -206,39 +205,41 @@ impl BallistaContext { Ok(()) } - pub fn register_csv( + pub async fn register_csv( &self, name: &str, path: &str, - options: CsvReadOptions, + options: CsvReadOptions<'_>, ) -> Result<()> { - match self.read_csv(path, options)?.to_logical_plan() { + match self.read_csv(path, options).await?.to_logical_plan() { LogicalPlan::TableScan { source, .. } => self.register_table(name, source), _ => Err(DataFusionError::Internal("Expected tables scan".to_owned())), } } - pub fn register_parquet(&self, name: &str, path: &str) -> Result<()> { - match self.read_parquet(path)?.to_logical_plan() { + pub async fn register_parquet(&self, name: &str, path: &str) -> Result<()> { + match self.read_parquet(path).await?.to_logical_plan() { LogicalPlan::TableScan { source, .. } => self.register_table(name, source), _ => Err(DataFusionError::Internal("Expected tables scan".to_owned())), } } - pub fn register_avro( + pub async fn register_avro( &self, name: &str, path: &str, - options: AvroReadOptions, + options: AvroReadOptions<'_>, ) -> Result<()> { - match self.read_avro(path, options)?.to_logical_plan() { + match self.read_avro(path, options).await?.to_logical_plan() { LogicalPlan::TableScan { source, .. } => self.register_table(name, source), _ => Err(DataFusionError::Internal("Expected tables scan".to_owned())), } } - /// Create a DataFrame from a SQL statement - pub fn sql(&self, sql: &str) -> Result> { + /// Create a DataFrame from a SQL statement. + /// + /// Async because CreateExternalTable might require to resolve the schema + pub async fn sql(&self, sql: &str) -> Result> { let mut ctx = { let state = self.state.lock().unwrap(); create_df_ctx_with_ballista_query_planner( @@ -275,15 +276,17 @@ impl BallistaContext { CsvReadOptions::new() .schema(&schema.as_ref().to_owned().into()) .has_header(*has_header), - )?; + ) + .await?; Ok(Arc::new(DataFrameImpl::new(ctx.state, &plan))) } FileType::Parquet => { - self.register_parquet(name, location)?; + self.register_parquet(name, location).await?; Ok(Arc::new(DataFrameImpl::new(ctx.state, &plan))) } FileType::Avro => { - self.register_avro(name, location, AvroReadOptions::default())?; + self.register_avro(name, location, AvroReadOptions::default()) + .await?; Ok(Arc::new(DataFrameImpl::new(ctx.state, &plan))) } _ => Err(DataFusionError::NotImplemented(format!( @@ -292,7 +295,7 @@ impl BallistaContext { ))), }, - _ => ctx.sql(sql), + _ => ctx.sql(sql).await, } } } @@ -306,7 +309,7 @@ mod tests { let context = BallistaContext::standalone(&BallistaConfig::new().unwrap(), 1) .await .unwrap(); - let df = context.sql("SELECT 1;").unwrap(); + let df = context.sql("SELECT 1;").await.unwrap(); df.collect().await.unwrap(); } } diff --git a/ballista/rust/core/Cargo.toml b/ballista/rust/core/Cargo.toml index 94ee71d6d1c4..ac53aa00e47e 100644 --- a/ballista/rust/core/Cargo.toml +++ b/ballista/rust/core/Cargo.toml @@ -41,6 +41,7 @@ sqlparser = "0.11.0" tokio = "1.0" tonic = "0.5" uuid = { version = "0.8", features = ["v4"] } +chrono = "0.4" arrow-flight = { version = "^5.3" } diff --git a/ballista/rust/core/proto/ballista.proto b/ballista/rust/core/proto/ballista.proto index 9a2ec710411b..8d9488dc8f49 100644 --- a/ballista/rust/core/proto/ballista.proto +++ b/ballista/rust/core/proto/ballista.proto @@ -20,6 +20,8 @@ syntax = "proto3"; package ballista.protobuf; +import "google/protobuf/timestamp.proto"; + option java_multiple_files = true; option java_package = "org.ballistacompute.protobuf"; option java_outer_classname = "BallistaProto"; @@ -241,8 +243,7 @@ message SortExprNode { // LogicalPlan is a nested type message LogicalPlanNode { oneof LogicalPlanType { - CsvTableScanNode csv_scan = 1; - ParquetTableScanNode parquet_scan = 2; + ListingTableScanNode listing_scan = 1; ProjectionNode projection = 3; SelectionNode selection = 4; LimitNode limit = 5; @@ -256,7 +257,6 @@ message LogicalPlanNode { WindowNode window = 13; AnalyzeNode analyze = 14; CrossJoinNode cross_join = 15; - AvroTableScanNode avro_scan = 16; } } @@ -264,17 +264,6 @@ message ProjectionColumns { repeated string columns = 1; } -message CsvTableScanNode { - string table_name = 1; - string path = 2; - bool has_header = 3; - string delimiter = 4; - string file_extension = 5; - ProjectionColumns projection = 6; - Schema schema = 7; - repeated LogicalExprNode filters = 8; -} - message Statistics { int64 num_rows = 1; int64 total_byte_size = 2; @@ -284,30 +273,36 @@ message Statistics { message PartitionedFile { string path = 1; - Statistics statistics = 2; + uint64 size = 2; + uint64 last_modified_ns = 3; } -message TableDescriptor { - string path = 1; - repeated PartitionedFile partition_files = 2; - Schema schema = 3; +message CsvFormat { + bool has_header = 1; + string delimiter = 2; } -message ParquetTableScanNode { - string table_name = 1; - TableDescriptor table_desc = 2; - ProjectionColumns projection = 3; - repeated LogicalExprNode filters = 4; - uint32 target_partitions = 5; +message ParquetFormat { + bool enable_pruning = 1; } -message AvroTableScanNode { +message AvroFormat {} + +message ListingTableScanNode { string table_name = 1; string path = 2; string file_extension = 3; ProjectionColumns projection = 4; Schema schema = 5; repeated LogicalExprNode filters = 6; + repeated string partitions = 7; + bool collect_stat = 8; + uint32 target_partitions = 9; + oneof FileFormatType { + CsvFormat csv = 10; + ParquetFormat parquet = 11; + AvroFormat avro = 12; + } } message ProjectionNode { @@ -603,40 +598,42 @@ message FilterExecNode { PhysicalExprNode expr = 2; } -message ParquetPartition { - uint32 index = 1; - repeated PartitionedFile files = 2; +message FilePartition { + repeated PartitionedFile files = 1; +} + +message ScanLimit { + // wrap into a message to make it optional + uint32 limit = 1; } message ParquetScanExecNode { - repeated ParquetPartition partitions = 1; + repeated FilePartition partitions = 1; Schema schema = 2; - repeated uint32 projection = 3; uint32 batch_size = 4; + repeated uint32 projection = 6; + ScanLimit limit = 7; + Statistics statistics = 8; } message CsvScanExecNode { - string path = 1; - repeated uint32 projection = 2; - Schema schema = 3; - string file_extension = 4; - bool has_header = 5; - uint32 batch_size = 6; - string delimiter = 7; - - // partition filenames - repeated string filename = 8; + repeated PartitionedFile files = 1; + Schema schema = 2; + bool has_header = 3; + uint32 batch_size = 4; + string delimiter = 5; + repeated uint32 projection = 6; + ScanLimit limit = 7; + Statistics statistics = 8; } message AvroScanExecNode { - string path = 1; - repeated uint32 projection = 2; - Schema schema = 3; - string file_extension = 4; - uint32 batch_size = 5; - - // partition filenames - repeated string filename = 8; + repeated PartitionedFile files = 1; + Schema schema = 2; + uint32 batch_size = 4; + repeated uint32 projection = 6; + ScanLimit limit = 7; + Statistics statistics = 8; } enum PartitionMode { @@ -951,7 +948,6 @@ message GetFileMetadataParams { message GetFileMetadataResult { Schema schema = 1; - repeated FilePartitionMetadata partitions = 2; } message FilePartitionMetadata { diff --git a/ballista/rust/core/src/serde/logical_plan/from_proto.rs b/ballista/rust/core/src/serde/logical_plan/from_proto.rs index 353be9a59642..07eced784004 100644 --- a/ballista/rust/core/src/serde/logical_plan/from_proto.rs +++ b/ballista/rust/core/src/serde/logical_plan/from_proto.rs @@ -18,11 +18,16 @@ //! Serde code to convert from protocol buffers to Rust data structures. use crate::error::BallistaError; -use crate::serde::{from_proto_binary_op, proto_error, protobuf}; +use crate::serde::{from_proto_binary_op, proto_error, protobuf, str_to_byte}; use crate::{convert_box_required, convert_required}; use datafusion::arrow::datatypes::{DataType, Field, Schema, TimeUnit}; -use datafusion::datasource::parquet::{ParquetTable, ParquetTableDescriptor}; -use datafusion::datasource::{PartitionedFile, TableDescriptor}; +use datafusion::datasource::file_format::avro::AvroFormat; +use datafusion::datasource::file_format::csv::CsvFormat; +use datafusion::datasource::file_format::parquet::ParquetFormat; +use datafusion::datasource::file_format::FileFormat; +use datafusion::datasource::listing::{ListingOptions, ListingTable}; +use datafusion::datasource::object_store::local::LocalFileSystem; +use datafusion::datasource::object_store::{FileMeta, SizedFile}; use datafusion::logical_plan::window_frames::{ WindowFrame, WindowFrameBound, WindowFrameUnits, }; @@ -32,10 +37,10 @@ use datafusion::logical_plan::{ LogicalPlan, LogicalPlanBuilder, Operator, }; use datafusion::physical_plan::aggregates::AggregateFunction; -use datafusion::physical_plan::avro::AvroReadOptions; -use datafusion::physical_plan::csv::CsvReadOptions; use datafusion::physical_plan::window_functions::BuiltInWindowFunction; +use datafusion::prelude::*; use datafusion::scalar::ScalarValue; +use protobuf::listing_table_scan_node::FileFormatType; use protobuf::logical_plan_node::LogicalPlanType; use protobuf::{logical_expr_node::ExprType, scalar_type}; use std::{ @@ -116,13 +121,8 @@ impl TryInto for &protobuf::LogicalPlanNode { .build() .map_err(|e| e.into()) } - LogicalPlanType::CsvScan(scan) => { + LogicalPlanType::ListingScan(scan) => { let schema: Schema = convert_required!(scan.schema)?; - let options = CsvReadOptions::new() - .schema(&schema) - .delimiter(scan.delimiter.as_bytes()[0]) - .file_extension(&scan.file_extension) - .has_header(scan.has_header); let mut projection = None; if let Some(columns) = &scan.projection { @@ -134,73 +134,55 @@ impl TryInto for &protobuf::LogicalPlanNode { projection = Some(column_indices); } - LogicalPlanBuilder::scan_csv_with_name( - &scan.path, - options, - projection, - &scan.table_name, - )? - .build() - .map_err(|e| e.into()) - } - LogicalPlanType::ParquetScan(scan) => { - let descriptor: TableDescriptor = convert_required!(scan.table_desc)?; - let projection = match scan.projection.as_ref() { - None => None, - Some(columns) => { - let schema = descriptor.schema.clone(); - let r: Result, _> = columns - .columns - .iter() - .map(|col_name| { - schema.fields().iter().position(|field| field.name() == col_name).ok_or_else(|| { - let column_names: Vec<&String> = schema.fields().iter().map(|f| f.name()).collect(); - proto_error(format!( - "Parquet projection contains column name that is not present in schema. Column name: {}. Schema columns: {:?}", - col_name, column_names - )) - }) - }) - .collect(); - Some(r?) - } - }; + let filters = scan + .filters + .iter() + .map(|e| e.try_into()) + .collect::, _>>()?; - let parquet_table = ParquetTable::try_new_with_desc( - Arc::new(ParquetTableDescriptor { descriptor }), - scan.target_partitions as usize, - true, - )?; - LogicalPlanBuilder::scan( - &scan.table_name, - Arc::new(parquet_table), - projection, - )? - .build() - .map_err(|e| e.into()) - } - LogicalPlanType::AvroScan(scan) => { - let schema: Schema = convert_required!(scan.schema)?; - let options = AvroReadOptions { - schema: Some(Arc::new(schema.clone())), - file_extension: &scan.file_extension, + let file_format: Arc = + match scan.file_format_type.as_ref().ok_or_else(|| { + proto_error(format!( + "logical_plan::from_proto() Unsupported file format '{:?}'", + self + )) + })? { + &FileFormatType::Parquet(protobuf::ParquetFormat { + enable_pruning, + }) => Arc::new( + ParquetFormat::default().with_enable_pruning(enable_pruning), + ), + FileFormatType::Csv(protobuf::CsvFormat { + has_header, + delimiter, + }) => Arc::new( + CsvFormat::default() + .with_has_header(*has_header) + .with_delimiter(str_to_byte(delimiter)?), + ), + FileFormatType::Avro(..) => Arc::new(AvroFormat::default()), + }; + + let options = ListingOptions { + file_extension: scan.file_extension.clone(), + format: file_format, + partitions: scan.partitions.clone(), + collect_stat: scan.collect_stat, + target_partitions: scan.target_partitions as usize, }; - let mut projection = None; - if let Some(columns) = &scan.projection { - let column_indices = columns - .columns - .iter() - .map(|name| schema.index_of(name)) - .collect::, _>>()?; - projection = Some(column_indices); - } - - LogicalPlanBuilder::scan_avro_with_name( - &scan.path, + let provider = ListingTable::new( + Arc::new(LocalFileSystem {}), + scan.path.clone(), + Arc::new(schema), options, - projection, + ); + + LogicalPlanBuilder::scan_with_filters( &scan.table_name, + Arc::new(provider), + projection, + filters, )? .build() .map_err(|e| e.into()) @@ -343,61 +325,6 @@ impl TryInto for &protobuf::LogicalPlanNode { } } -impl TryInto for &protobuf::TableDescriptor { - type Error = BallistaError; - - fn try_into(self) -> Result { - let partition_files = self - .partition_files - .iter() - .map(|f| f.try_into()) - .collect::, _>>()?; - let schema = convert_required!(self.schema)?; - Ok(TableDescriptor { - path: self.path.to_owned(), - partition_files, - schema: Arc::new(schema), - }) - } -} - -impl TryInto for &protobuf::PartitionedFile { - type Error = BallistaError; - - fn try_into(self) -> Result { - let statistics = convert_required!(self.statistics)?; - Ok(PartitionedFile { - path: self.path.clone(), - statistics, - }) - } -} - -impl From<&protobuf::ColumnStats> for ColumnStatistics { - fn from(cs: &protobuf::ColumnStats) -> ColumnStatistics { - ColumnStatistics { - null_count: Some(cs.null_count as usize), - max_value: cs.max_value.as_ref().map(|m| m.try_into().unwrap()), - min_value: cs.min_value.as_ref().map(|m| m.try_into().unwrap()), - distinct_count: Some(cs.distinct_count as usize), - } - } -} - -impl TryInto for &protobuf::Statistics { - type Error = BallistaError; - - fn try_into(self) -> Result { - let column_statistics = self.column_stats.iter().map(|s| s.into()).collect(); - Ok(Statistics { - num_rows: Some(self.num_rows as usize), - total_byte_size: Some(self.total_byte_size as usize), - column_statistics: Some(column_statistics), - is_exact: self.is_exact, - }) - } -} - impl From<&protobuf::Column> for Column { fn from(c: &protobuf::Column) -> Column { let c = c.clone(); @@ -1215,7 +1142,7 @@ impl TryInto for &protobuf::Field { } use crate::serde::protobuf::ColumnStats; -use datafusion::physical_plan::{aggregates, windows, ColumnStatistics, Statistics}; +use datafusion::physical_plan::{aggregates, windows}; use datafusion::prelude::{ array, date_part, date_trunc, length, lower, ltrim, md5, rtrim, sha224, sha256, sha384, sha512, trim, upper, diff --git a/ballista/rust/core/src/serde/logical_plan/mod.rs b/ballista/rust/core/src/serde/logical_plan/mod.rs index ada3c85de674..09bcf1fd048f 100644 --- a/ballista/rust/core/src/serde/logical_plan/mod.rs +++ b/ballista/rust/core/src/serde/logical_plan/mod.rs @@ -26,16 +26,17 @@ mod roundtrip_tests { use core::panic; use datafusion::{ arrow::datatypes::{DataType, Field, IntervalUnit, Schema, TimeUnit}, + datasource::object_store::local::LocalFileSystem, logical_plan::{ col, Expr, LogicalPlan, LogicalPlanBuilder, Partitioning, ToDFSchema, }, - physical_plan::{csv::CsvReadOptions, functions::BuiltinScalarFunction::Sqrt}, + physical_plan::functions::BuiltinScalarFunction::Sqrt, prelude::*, scalar::ScalarValue, sql::parser::FileType, }; use protobuf::arrow_type; - use std::convert::TryInto; + use std::{convert::TryInto, sync::Arc}; //Given a identity of a LogicalPlan converts it to protobuf and back, using debug formatting to test equality. macro_rules! roundtrip_test { @@ -57,8 +58,8 @@ mod roundtrip_tests { }; } - #[test] - fn roundtrip_repartition() -> Result<()> { + #[tokio::test] + async fn roundtrip_repartition() -> Result<()> { use datafusion::logical_plan::Partitioning; let test_batch_sizes = [usize::MIN, usize::MAX, 43256]; @@ -76,10 +77,13 @@ mod roundtrip_tests { let plan = std::sync::Arc::new( LogicalPlanBuilder::scan_csv( + Arc::new(LocalFileSystem {}), "employee.csv", CsvReadOptions::new().schema(&schema).has_header(true), Some(vec![3, 4]), + 4, ) + .await .and_then(|plan| plan.sort(vec![col("salary")])) .and_then(|plan| plan.build()) .map_err(BallistaError::DataFusionError)?, @@ -665,8 +669,8 @@ mod roundtrip_tests { Ok(()) } - #[test] - fn roundtrip_analyze() -> Result<()> { + #[tokio::test] + async fn roundtrip_analyze() -> Result<()> { let schema = Schema::new(vec![ Field::new("id", DataType::Int32, false), Field::new("first_name", DataType::Utf8, false), @@ -676,20 +680,26 @@ mod roundtrip_tests { ]); let verbose_plan = LogicalPlanBuilder::scan_csv( + Arc::new(LocalFileSystem {}), "employee.csv", CsvReadOptions::new().schema(&schema).has_header(true), Some(vec![3, 4]), + 4, ) + .await .and_then(|plan| plan.sort(vec![col("salary")])) .and_then(|plan| plan.explain(true, true)) .and_then(|plan| plan.build()) .map_err(BallistaError::DataFusionError)?; let plan = LogicalPlanBuilder::scan_csv( + Arc::new(LocalFileSystem {}), "employee.csv", CsvReadOptions::new().schema(&schema).has_header(true), Some(vec![3, 4]), + 4, ) + .await .and_then(|plan| plan.sort(vec![col("salary")])) .and_then(|plan| plan.explain(false, true)) .and_then(|plan| plan.build()) @@ -702,8 +712,8 @@ mod roundtrip_tests { Ok(()) } - #[test] - fn roundtrip_explain() -> Result<()> { + #[tokio::test] + async fn roundtrip_explain() -> Result<()> { let schema = Schema::new(vec![ Field::new("id", DataType::Int32, false), Field::new("first_name", DataType::Utf8, false), @@ -713,20 +723,26 @@ mod roundtrip_tests { ]); let verbose_plan = LogicalPlanBuilder::scan_csv( + Arc::new(LocalFileSystem {}), "employee.csv", CsvReadOptions::new().schema(&schema).has_header(true), Some(vec![3, 4]), + 4, ) + .await .and_then(|plan| plan.sort(vec![col("salary")])) .and_then(|plan| plan.explain(true, false)) .and_then(|plan| plan.build()) .map_err(BallistaError::DataFusionError)?; let plan = LogicalPlanBuilder::scan_csv( + Arc::new(LocalFileSystem {}), "employee.csv", CsvReadOptions::new().schema(&schema).has_header(true), Some(vec![3, 4]), + 4, ) + .await .and_then(|plan| plan.sort(vec![col("salary")])) .and_then(|plan| plan.explain(false, false)) .and_then(|plan| plan.build()) @@ -739,8 +755,8 @@ mod roundtrip_tests { Ok(()) } - #[test] - fn roundtrip_join() -> Result<()> { + #[tokio::test] + async fn roundtrip_join() -> Result<()> { let schema = Schema::new(vec![ Field::new("id", DataType::Int32, false), Field::new("first_name", DataType::Utf8, false), @@ -750,18 +766,24 @@ mod roundtrip_tests { ]); let scan_plan = LogicalPlanBuilder::scan_csv( + Arc::new(LocalFileSystem {}), "employee1", CsvReadOptions::new().schema(&schema).has_header(true), Some(vec![0, 3, 4]), - )? + 4, + ) + .await? .build() .map_err(BallistaError::DataFusionError)?; let plan = LogicalPlanBuilder::scan_csv( + Arc::new(LocalFileSystem {}), "employee2", CsvReadOptions::new().schema(&schema).has_header(true), Some(vec![0, 3, 4]), + 4, ) + .await .and_then(|plan| plan.join(&scan_plan, JoinType::Inner, (vec!["id"], vec!["id"]))) .and_then(|plan| plan.build()) .map_err(BallistaError::DataFusionError)?; @@ -770,8 +792,8 @@ mod roundtrip_tests { Ok(()) } - #[test] - fn roundtrip_sort() -> Result<()> { + #[tokio::test] + async fn roundtrip_sort() -> Result<()> { let schema = Schema::new(vec![ Field::new("id", DataType::Int32, false), Field::new("first_name", DataType::Utf8, false), @@ -781,10 +803,13 @@ mod roundtrip_tests { ]); let plan = LogicalPlanBuilder::scan_csv( + Arc::new(LocalFileSystem {}), "employee.csv", CsvReadOptions::new().schema(&schema).has_header(true), Some(vec![3, 4]), + 4, ) + .await .and_then(|plan| plan.sort(vec![col("salary")])) .and_then(|plan| plan.build()) .map_err(BallistaError::DataFusionError)?; @@ -793,8 +818,8 @@ mod roundtrip_tests { Ok(()) } - #[test] - fn roundtrip_empty_relation() -> Result<()> { + #[tokio::test] + async fn roundtrip_empty_relation() -> Result<()> { let plan_false = LogicalPlanBuilder::empty(false) .build() .map_err(BallistaError::DataFusionError)?; @@ -810,8 +835,8 @@ mod roundtrip_tests { Ok(()) } - #[test] - fn roundtrip_logical_plan() -> Result<()> { + #[tokio::test] + async fn roundtrip_logical_plan() -> Result<()> { let schema = Schema::new(vec![ Field::new("id", DataType::Int32, false), Field::new("first_name", DataType::Utf8, false), @@ -821,10 +846,13 @@ mod roundtrip_tests { ]); let plan = LogicalPlanBuilder::scan_csv( + Arc::new(LocalFileSystem {}), "employee.csv", CsvReadOptions::new().schema(&schema).has_header(true), Some(vec![3, 4]), + 4, ) + .await .and_then(|plan| plan.aggregate(vec![col("state")], vec![max(col("salary"))])) .and_then(|plan| plan.build()) .map_err(BallistaError::DataFusionError)?; diff --git a/ballista/rust/core/src/serde/logical_plan/to_proto.rs b/ballista/rust/core/src/serde/logical_plan/to_proto.rs index 402422adb205..ba7daca18a4e 100644 --- a/ballista/rust/core/src/serde/logical_plan/to_proto.rs +++ b/ballista/rust/core/src/serde/logical_plan/to_proto.rs @@ -20,13 +20,18 @@ //! processes. use super::super::proto_error; -use crate::serde::{protobuf, BallistaError}; +use crate::serde::{byte_to_string, protobuf, BallistaError}; use datafusion::arrow::datatypes::{ DataType, Field, IntervalUnit, Schema, SchemaRef, TimeUnit, }; -use datafusion::datasource::avro::AvroFile; -use datafusion::datasource::{CsvFile, PartitionedFile, TableDescriptor}; +use datafusion::datasource::file_format::avro::AvroFormat; +use datafusion::datasource::file_format::csv::CsvFormat; +use datafusion::datasource::TableProvider; + +use datafusion::datasource::file_format::parquet::ParquetFormat; +use datafusion::datasource::listing::ListingTable; use datafusion::logical_plan::{ + exprlist_to_fields, window_frames::{WindowFrame, WindowFrameBound, WindowFrameUnits}, Column, Expr, JoinConstraint, JoinType, LogicalPlan, }; @@ -36,7 +41,7 @@ use datafusion::physical_plan::window_functions::{ BuiltInWindowFunction, WindowFunction, }; use datafusion::physical_plan::{ColumnStatistics, Statistics}; -use datafusion::{datasource::parquet::ParquetTable, logical_plan::exprlist_to_fields}; +use protobuf::listing_table_scan_node::FileFormatType; use protobuf::{ arrow_type, logical_expr_node::ExprType, scalar_type, DateUnit, PrimitiveScalarType, ScalarListValue, ScalarType, @@ -256,59 +261,6 @@ impl TryInto for &protobuf::ArrowType { } } -impl From<&ColumnStatistics> for protobuf::ColumnStats { - fn from(cs: &ColumnStatistics) -> protobuf::ColumnStats { - protobuf::ColumnStats { - min_value: cs.min_value.as_ref().map(|m| m.try_into().unwrap()), - max_value: cs.max_value.as_ref().map(|m| m.try_into().unwrap()), - null_count: cs.null_count.map(|n| n as u32).unwrap_or(0), - distinct_count: cs.distinct_count.map(|n| n as u32).unwrap_or(0), - } - } -} - -impl From<&Statistics> for protobuf::Statistics { - fn from(s: &Statistics) -> protobuf::Statistics { - let none_value = -1_i64; - let column_stats = match &s.column_statistics { - None => vec![], - Some(column_stats) => column_stats.iter().map(|s| s.into()).collect(), - }; - protobuf::Statistics { - num_rows: s.num_rows.map(|n| n as i64).unwrap_or(none_value), - total_byte_size: s.total_byte_size.map(|n| n as i64).unwrap_or(none_value), - column_stats, - is_exact: s.is_exact, - } - } -} - -impl From<&PartitionedFile> for protobuf::PartitionedFile { - fn from(pf: &PartitionedFile) -> protobuf::PartitionedFile { - protobuf::PartitionedFile { - path: pf.path.clone(), - statistics: Some((&pf.statistics).into()), - } - } -} - -impl TryFrom for protobuf::TableDescriptor { - type Error = BallistaError; - - fn try_from(desc: TableDescriptor) -> Result { - let partition_files: Vec = - desc.partition_files.iter().map(|pf| pf.into()).collect(); - - let schema: protobuf::Schema = desc.schema.into(); - - Ok(protobuf::TableDescriptor { - path: desc.path, - partition_files, - schema: Some(schema), - }) - } -} - impl TryInto for &Box { type Error = BallistaError; fn try_into(self) -> Result { @@ -748,49 +700,46 @@ impl TryInto for &LogicalPlan { .map(|filter| filter.try_into()) .collect::, _>>()?; - if let Some(parquet) = source.downcast_ref::() { - let table_desc: protobuf::TableDescriptor = - parquet.desc.descriptor.clone().try_into()?; - Ok(protobuf::LogicalPlanNode { - logical_plan_type: Some(LogicalPlanType::ParquetScan( - protobuf::ParquetTableScanNode { - table_name: table_name.to_owned(), - table_desc: Some(table_desc), - projection, - filters, - target_partitions: parquet.get_target_partitions() as u32, - }, - )), - }) - } else if let Some(csv) = source.downcast_ref::() { - let delimiter = [csv.delimiter()]; - let delimiter = std::str::from_utf8(&delimiter).map_err(|_| { - BallistaError::General("Invalid CSV delimiter".to_owned()) - })?; + if let Some(listing_table) = source.downcast_ref::() { + let any = listing_table.options().format.as_any(); + let file_format_type = if let Some(parquet) = + any.downcast_ref::() + { + FileFormatType::Parquet(protobuf::ParquetFormat { + enable_pruning: parquet.enable_pruning(), + }) + } else if let Some(csv) = any.downcast_ref::() { + FileFormatType::Csv(protobuf::CsvFormat { + delimiter: byte_to_string(csv.delimiter())?, + has_header: csv.has_header(), + }) + } else if any.is::() { + FileFormatType::Avro(protobuf::AvroFormat {}) + } else { + return Err(proto_error(format!( + "Error converting file format, {:?} is invalid as a datafusion foramt.", + listing_table.options().format + ))); + }; Ok(protobuf::LogicalPlanNode { - logical_plan_type: Some(LogicalPlanType::CsvScan( - protobuf::CsvTableScanNode { + logical_plan_type: Some(LogicalPlanType::ListingScan( + protobuf::ListingTableScanNode { + file_format_type: Some(file_format_type), table_name: table_name.to_owned(), - path: csv.path().to_owned(), - projection, + collect_stat: listing_table.options().collect_stat, + file_extension: listing_table + .options() + .file_extension + .clone(), + partitions: listing_table.options().partitions.clone(), + path: listing_table.path().to_owned(), schema: Some(schema), - has_header: csv.has_header(), - delimiter: delimiter.to_string(), - file_extension: csv.file_extension().to_string(), - filters, - }, - )), - }) - } else if let Some(avro) = source.downcast_ref::() { - Ok(protobuf::LogicalPlanNode { - logical_plan_type: Some(LogicalPlanType::AvroScan( - protobuf::AvroTableScanNode { - table_name: table_name.to_owned(), - path: avro.path().to_owned(), projection, - schema: Some(schema), - file_extension: avro.file_extension().to_string(), filters, + target_partitions: listing_table + .options() + .target_partitions + as u32, }, )), }) diff --git a/ballista/rust/core/src/serde/mod.rs b/ballista/rust/core/src/serde/mod.rs index a4df5a45555d..4a32b24b9531 100644 --- a/ballista/rust/core/src/serde/mod.rs +++ b/ballista/rust/core/src/serde/mod.rs @@ -339,3 +339,17 @@ impl From for protobuf::JoinConstraint { } } } + +fn byte_to_string(b: u8) -> Result { + let b = &[b]; + let b = std::str::from_utf8(b) + .map_err(|_| BallistaError::General("Invalid CSV delimiter".to_owned()))?; + Ok(b.to_owned()) +} + +fn str_to_byte(s: &str) -> Result { + if s.len() != 1 { + return Err(BallistaError::General("Invalid CSV delimiter".to_owned())); + } + Ok(s.as_bytes()[0]) +} diff --git a/ballista/rust/core/src/serde/physical_plan/from_proto.rs b/ballista/rust/core/src/serde/physical_plan/from_proto.rs index 5241e8b2bd5e..680e41980c72 100644 --- a/ballista/rust/core/src/serde/physical_plan/from_proto.rs +++ b/ballista/rust/core/src/serde/physical_plan/from_proto.rs @@ -28,14 +28,16 @@ use crate::execution_plans::{ use crate::serde::protobuf::repartition_exec_node::PartitionMethod; use crate::serde::protobuf::ShuffleReaderPartition; use crate::serde::scheduler::PartitionLocation; -use crate::serde::{from_proto_binary_op, proto_error, protobuf}; +use crate::serde::{from_proto_binary_op, proto_error, protobuf, str_to_byte}; use crate::{convert_box_required, convert_required, into_required}; +use chrono::{DateTime, NaiveDateTime, TimeZone, Utc}; use datafusion::arrow::datatypes::{DataType, Schema, SchemaRef}; use datafusion::catalog::catalog::{ CatalogList, CatalogProvider, MemoryCatalogList, MemoryCatalogProvider, }; -use datafusion::datasource::object_store::ObjectStoreRegistry; -use datafusion::datasource::FilePartition; +use datafusion::datasource::object_store::local::LocalFileSystem; +use datafusion::datasource::object_store::{FileMeta, ObjectStoreRegistry, SizedFile}; +use datafusion::datasource::{FilePartition, PartitionedFile}; use datafusion::execution::context::{ ExecutionConfig, ExecutionContextState, ExecutionProps, }; @@ -43,12 +45,11 @@ use datafusion::logical_plan::{ window_frames::WindowFrame, DFSchema, Expr, JoinConstraint, JoinType, }; use datafusion::physical_plan::aggregates::{create_aggregate_expr, AggregateFunction}; -use datafusion::physical_plan::avro::{AvroExec, AvroReadOptions}; use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec; +use datafusion::physical_plan::file_format::{AvroExec, CsvExec, ParquetExec}; use datafusion::physical_plan::hash_aggregate::{AggregateMode, HashAggregateExec}; use datafusion::physical_plan::hash_join::PartitionMode; use datafusion::physical_plan::metrics::ExecutionPlanMetricsSet; -use datafusion::physical_plan::parquet::ParquetPartition; use datafusion::physical_plan::planner::DefaultPhysicalPlanner; use datafusion::physical_plan::window_functions::{ BuiltInWindowFunction, WindowFunction, @@ -57,7 +58,6 @@ use datafusion::physical_plan::windows::{create_window_expr, WindowAggExec}; use datafusion::physical_plan::{ coalesce_batches::CoalesceBatchesExec, cross_join::CrossJoinExec, - csv::CsvExec, empty::EmptyExec, expressions::{ col, Avg, BinaryExpr, CaseExpr, CastExpr, Column, InListExpr, IsNotNullExpr, @@ -68,14 +68,13 @@ use datafusion::physical_plan::{ functions::{self, BuiltinScalarFunction, ScalarFunctionExpr}, hash_join::HashJoinExec, limit::{GlobalLimitExec, LocalLimitExec}, - parquet::ParquetExec, projection::ProjectionExec, repartition::RepartitionExec, sort::{SortExec, SortOptions}, Partitioning, }; use datafusion::physical_plan::{ - AggregateExpr, ExecutionPlan, PhysicalExpr, Statistics, WindowExpr, + AggregateExpr, ColumnStatistics, ExecutionPlan, PhysicalExpr, Statistics, WindowExpr, }; use datafusion::prelude::CsvReadOptions; use log::debug; @@ -121,53 +120,64 @@ impl TryInto> for &protobuf::PhysicalPlanNode { } PhysicalPlanType::CsvScan(scan) => { let schema = Arc::new(convert_required!(scan.schema)?); - let options = CsvReadOptions::new() - .has_header(scan.has_header) - .file_extension(&scan.file_extension) - .delimiter(scan.delimiter.as_bytes()[0]) - .schema(&schema); let projection = scan.projection.iter().map(|i| *i as usize).collect(); - Ok(Arc::new(CsvExec::try_new( - &scan.path, - options, + let statistics = convert_required!(scan.statistics)?; + + Ok(Arc::new(CsvExec::new( + Arc::new(LocalFileSystem {}), + scan.files + .iter() + .map(|f| f.try_into()) + .collect::, _>>()?, + statistics, + schema, + scan.has_header, + str_to_byte(&scan.delimiter)?, Some(projection), scan.batch_size as usize, - None, - )?)) + scan.limit.as_ref().map(|sl| sl.limit as usize), + ))) } PhysicalPlanType::ParquetScan(scan) => { - let partitions = scan - .partitions - .iter() - .map(|p| p.try_into()) - .collect::, _>>()?; let schema = Arc::new(convert_required!(scan.schema)?); let projection = scan.projection.iter().map(|i| *i as usize).collect(); + let statistics = convert_required!(scan.statistics)?; + Ok(Arc::new(ParquetExec::new( - partitions, + Arc::new(LocalFileSystem {}), + scan.partitions + .iter() + .map(|p| { + let it = p.files.iter().map(|f| f.try_into()); + it.collect::, _>>() + }) + .collect::>, _>>()?, + statistics, schema, Some(projection), - Statistics::default(), - ExecutionPlanMetricsSet::new(), + // TODO predicate should be de-serialized None, scan.batch_size as usize, - None, + scan.limit.as_ref().map(|sl| sl.limit as usize), ))) } PhysicalPlanType::AvroScan(scan) => { let schema = Arc::new(convert_required!(scan.schema)?); - let options = AvroReadOptions { - schema: Some(schema), - file_extension: &scan.file_extension, - }; let projection = scan.projection.iter().map(|i| *i as usize).collect(); - Ok(Arc::new(AvroExec::try_from_path( - &scan.path, - options, + let statistics = convert_required!(scan.statistics)?; + + Ok(Arc::new(AvroExec::new( + Arc::new(LocalFileSystem {}), + scan.files + .iter() + .map(|f| f.try_into()) + .collect::, _>>()?, + statistics, + schema, Some(projection), scan.batch_size as usize, - None, - )?)) + scan.limit.as_ref().map(|sl| sl.limit as usize), + ))) } PhysicalPlanType::CoalesceBatches(coalesce_batches) => { let input: Arc = @@ -498,23 +508,6 @@ impl TryInto> for &protobuf::PhysicalPlanNode { } } -impl TryInto for &protobuf::ParquetPartition { - type Error = BallistaError; - - fn try_into(self) -> Result { - let files = self - .files - .iter() - .map(|f| f.try_into()) - .collect::, _>>()?; - Ok(ParquetPartition::new( - files, - self.index as usize, - ExecutionPlanMetricsSet::new(), - )) - } -} - impl From<&protobuf::PhysicalColumn> for Column { fn from(c: &protobuf::PhysicalColumn) -> Column { Column::new(&c.name, c.index as usize) @@ -747,3 +740,57 @@ pub fn parse_protobuf_hash_partitioning( None => Ok(None), } } + +impl TryInto for &protobuf::PartitionedFile { + type Error = BallistaError; + + fn try_into(self) -> Result { + Ok(PartitionedFile { + file_meta: FileMeta { + sized_file: SizedFile { + path: self.path.clone(), + size: self.size, + }, + last_modified: if self.last_modified_ns == 0 { + None + } else { + Some(Utc.timestamp_nanos(self.last_modified_ns as i64)) + }, + }, + }) + } +} + +impl From<&protobuf::ColumnStats> for ColumnStatistics { + fn from(cs: &protobuf::ColumnStats) -> ColumnStatistics { + ColumnStatistics { + null_count: Some(cs.null_count as usize), + max_value: cs.max_value.as_ref().map(|m| m.try_into().unwrap()), + min_value: cs.min_value.as_ref().map(|m| m.try_into().unwrap()), + distinct_count: Some(cs.distinct_count as usize), + } + } +} + +impl TryInto for &protobuf::Statistics { + type Error = BallistaError; + + fn try_into(self) -> Result { + let column_statistics = self + .column_stats + .iter() + .map(|s| s.into()) + .collect::>(); + Ok(Statistics { + num_rows: Some(self.num_rows as usize), + total_byte_size: Some(self.total_byte_size as usize), + // No column statistic (None) is encoded with empty array + column_statistics: if column_statistics.is_empty() { + None + } else { + Some(column_statistics) + }, + is_exact: self.is_exact, + }) + } +} diff --git a/ballista/rust/core/src/serde/physical_plan/to_proto.rs b/ballista/rust/core/src/serde/physical_plan/to_proto.rs index 22a49cb881ba..c52cbbbb957d 100644 --- a/ballista/rust/core/src/serde/physical_plan/to_proto.rs +++ b/ballista/rust/core/src/serde/physical_plan/to_proto.rs @@ -26,21 +26,27 @@ use std::{ sync::Arc, }; -use datafusion::logical_plan::JoinType; -use datafusion::physical_plan::coalesce_batches::CoalesceBatchesExec; -use datafusion::physical_plan::cross_join::CrossJoinExec; -use datafusion::physical_plan::csv::CsvExec; -use datafusion::physical_plan::expressions::{ - CaseExpr, InListExpr, IsNotNullExpr, IsNullExpr, NegativeExpr, NotExpr, -}; -use datafusion::physical_plan::expressions::{CastExpr, TryCastExpr}; -use datafusion::physical_plan::filter::FilterExec; use datafusion::physical_plan::hash_aggregate::AggregateMode; use datafusion::physical_plan::hash_join::{HashJoinExec, PartitionMode}; use datafusion::physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; -use datafusion::physical_plan::parquet::{ParquetExec, ParquetPartition}; use datafusion::physical_plan::projection::ProjectionExec; use datafusion::physical_plan::sort::SortExec; +use datafusion::physical_plan::{cross_join::CrossJoinExec, ColumnStatistics}; +use datafusion::physical_plan::{ + expressions::{ + CaseExpr, InListExpr, IsNotNullExpr, IsNullExpr, NegativeExpr, NotExpr, + }, + Statistics, +}; +use datafusion::physical_plan::{ + expressions::{CastExpr, TryCastExpr}, + file_format::ParquetExec, +}; +use datafusion::physical_plan::{file_format::AvroExec, filter::FilterExec}; +use datafusion::{ + datasource::PartitionedFile, physical_plan::coalesce_batches::CoalesceBatchesExec, +}; +use datafusion::{logical_plan::JoinType, physical_plan::file_format::CsvExec}; use datafusion::{ physical_plan::expressions::{Count, Literal}, scalar::ScalarValue, @@ -56,13 +62,13 @@ use datafusion::physical_plan::{AggregateExpr, ExecutionPlan, PhysicalExpr}; use datafusion::physical_plan::hash_aggregate::HashAggregateExec; use protobuf::physical_plan_node::PhysicalPlanType; -use crate::execution_plans::{ - ShuffleReaderExec, ShuffleWriterExec, UnresolvedShuffleExec, -}; use crate::serde::protobuf::repartition_exec_node::PartitionMethod; use crate::serde::scheduler::PartitionLocation; use crate::serde::{protobuf, BallistaError}; -use datafusion::physical_plan::avro::AvroExec; +use crate::{ + execution_plans::{ShuffleReaderExec, ShuffleWriterExec, UnresolvedShuffleExec}, + serde::byte_to_string, +}; use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion::physical_plan::functions::{BuiltinScalarFunction, ScalarFunctionExpr}; use datafusion::physical_plan::repartition::RepartitionExec; @@ -238,20 +244,21 @@ impl TryInto for Arc { ))), }) } else if let Some(exec) = plan.downcast_ref::() { - let delimiter = [*exec.delimiter().ok_or_else(|| { - BallistaError::General("Delimeter is not set for CsvExec".to_owned()) - })?]; - let delimiter = std::str::from_utf8(&delimiter).map_err(|_| { - BallistaError::General("Invalid CSV delimiter".to_owned()) - })?; - Ok(protobuf::PhysicalPlanNode { physical_plan_type: Some(PhysicalPlanType::CsvScan( protobuf::CsvScanExecNode { - path: exec.path().to_owned(), - filename: exec.filenames().to_vec(), + files: exec + .files() + .iter() + .map(|f| f.into()) + .collect::>(), + statistics: Some((&exec.statistics()).into()), + limit: exec + .limit() + .map(|l| protobuf::ScanLimit { limit: l as u32 }), projection: exec .projection() + .as_ref() .ok_or_else(|| { BallistaError::General( "projection in CsvExec dosn not exist.".to_owned(), @@ -260,22 +267,31 @@ impl TryInto for Arc { .iter() .map(|n| *n as u32) .collect(), - file_extension: exec.file_extension().to_owned(), schema: Some(exec.file_schema().as_ref().into()), has_header: exec.has_header(), - delimiter: delimiter.to_string(), + delimiter: byte_to_string(exec.delimiter())?, batch_size: exec.batch_size() as u32, }, )), }) } else if let Some(exec) = plan.downcast_ref::() { - let partitions = exec.partitions().iter().map(|p| p.into()).collect(); + let partitions = exec + .partitions() + .into_iter() + .map(|p| protobuf::FilePartition { + files: p.iter().map(|f| f.into()).collect(), + }) + .collect(); Ok(protobuf::PhysicalPlanNode { physical_plan_type: Some(PhysicalPlanType::ParquetScan( protobuf::ParquetScanExecNode { partitions, - schema: Some(exec.schema.as_ref().into()), + statistics: Some((&exec.statistics()).into()), + limit: exec + .limit() + .map(|l| protobuf::ScanLimit { limit: l as u32 }), + schema: Some(exec.schema().as_ref().into()), projection: exec .projection() .as_ref() @@ -290,19 +306,26 @@ impl TryInto for Arc { Ok(protobuf::PhysicalPlanNode { physical_plan_type: Some(PhysicalPlanType::AvroScan( protobuf::AvroScanExecNode { - path: exec.path().to_owned(), - filename: exec.filenames().to_vec(), + files: exec + .files() + .iter() + .map(|f| f.into()) + .collect::>(), + statistics: Some((&exec.statistics()).into()), + limit: exec + .limit() + .map(|l| protobuf::ScanLimit { limit: l as u32 }), projection: exec .projection() + .as_ref() .ok_or_else(|| { BallistaError::General( - "projection in AvroExec doesn't exist.".to_owned(), + "projection in AvroExec dosn not exist.".to_owned(), ) })? .iter() .map(|n| *n as u32) .collect(), - file_extension: exec.file_extension().to_owned(), schema: Some(exec.file_schema().as_ref().into()), batch_size: exec.batch_size() as u32, }, @@ -641,16 +664,6 @@ impl TryFrom> for protobuf::PhysicalExprNode { } } -impl From<&ParquetPartition> for protobuf::ParquetPartition { - fn from(p: &ParquetPartition) -> protobuf::ParquetPartition { - let files = p.file_partition.files.iter().map(|f| f.into()).collect(); - protobuf::ParquetPartition { - index: p.file_partition.index as u32, - files, - } - } -} - fn try_parse_when_then_expr( when_expr: &Arc, then_expr: &Arc, @@ -660,3 +673,44 @@ fn try_parse_when_then_expr( then_expr: Some(then_expr.clone().try_into()?), }) } + +impl From<&PartitionedFile> for protobuf::PartitionedFile { + fn from(pf: &PartitionedFile) -> protobuf::PartitionedFile { + protobuf::PartitionedFile { + path: pf.file_meta.path().to_owned(), + size: pf.file_meta.size(), + last_modified_ns: pf + .file_meta + .last_modified + .map(|ts| ts.timestamp_nanos() as u64) + .unwrap_or(0), + } + } +} + +impl From<&ColumnStatistics> for protobuf::ColumnStats { + fn from(cs: &ColumnStatistics) -> protobuf::ColumnStats { + protobuf::ColumnStats { + min_value: cs.min_value.as_ref().map(|m| m.try_into().unwrap()), + max_value: cs.max_value.as_ref().map(|m| m.try_into().unwrap()), + null_count: cs.null_count.map(|n| n as u32).unwrap_or(0), + distinct_count: cs.distinct_count.map(|n| n as u32).unwrap_or(0), + } + } +} + +impl From<&Statistics> for protobuf::Statistics { + fn from(s: &Statistics) -> protobuf::Statistics { + let none_value = -1_i64; + let column_stats = match &s.column_statistics { + None => vec![], + Some(column_stats) => column_stats.iter().map(|s| s.into()).collect(), + }; + protobuf::Statistics { + num_rows: s.num_rows.map(|n| n as i64).unwrap_or(none_value), + total_byte_size: s.total_byte_size.map(|n| n as i64).unwrap_or(none_value), + column_stats, + is_exact: s.is_exact, + } + } +} diff --git a/ballista/rust/core/src/utils.rs b/ballista/rust/core/src/utils.rs index fd12eb996785..80391b38355f 100644 --- a/ballista/rust/core/src/utils.rs +++ b/ballista/rust/core/src/utils.rs @@ -52,13 +52,12 @@ use datafusion::physical_optimizer::merge_exec::AddCoalescePartitionsExec; use datafusion::physical_optimizer::optimizer::PhysicalOptimizerRule; use datafusion::physical_plan::coalesce_batches::CoalesceBatchesExec; use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec; -use datafusion::physical_plan::csv::CsvExec; use datafusion::physical_plan::empty::EmptyExec; use datafusion::physical_plan::expressions::{BinaryExpr, Column, Literal}; +use datafusion::physical_plan::file_format::{CsvExec, ParquetExec}; use datafusion::physical_plan::filter::FilterExec; use datafusion::physical_plan::hash_aggregate::HashAggregateExec; use datafusion::physical_plan::hash_join::HashJoinExec; -use datafusion::physical_plan::parquet::ParquetExec; use datafusion::physical_plan::projection::ProjectionExec; use datafusion::physical_plan::sort::SortExec; use datafusion::physical_plan::{ diff --git a/ballista/rust/scheduler/src/lib.rs b/ballista/rust/scheduler/src/lib.rs index 47caf4c21ede..107ea28ff68b 100644 --- a/ballista/rust/scheduler/src/lib.rs +++ b/ballista/rust/scheduler/src/lib.rs @@ -22,6 +22,13 @@ pub mod planner; #[cfg(feature = "sled")] mod standalone; pub mod state; + +use datafusion::datasource::file_format::parquet::ParquetFormat; +use datafusion::datasource::file_format::FileFormat; +use datafusion::datasource::object_store::{local::LocalFileSystem, ObjectStore}; + +use futures::StreamExt; + #[cfg(feature = "sled")] pub use standalone::new_standalone_scheduler; @@ -40,10 +47,10 @@ use std::{fmt, net::IpAddr}; use ballista_core::serde::protobuf::{ execute_query_params::Query, executor_registration::OptionalHost, job_status, scheduler_grpc_server::SchedulerGrpc, task_status, ExecuteQueryParams, - ExecuteQueryResult, FailedJob, FilePartitionMetadata, FileType, - GetFileMetadataParams, GetFileMetadataResult, GetJobStatusParams, GetJobStatusResult, - JobStatus, PartitionId, PollWorkParams, PollWorkResult, QueuedJob, RunningJob, - TaskDefinition, TaskStatus, + ExecuteQueryResult, FailedJob, FileType, GetFileMetadataParams, + GetFileMetadataResult, GetJobStatusParams, GetJobStatusResult, JobStatus, + PartitionId, PollWorkParams, PollWorkResult, QueuedJob, RunningJob, TaskDefinition, + TaskStatus, }; use ballista_core::serde::scheduler::ExecutorMeta; @@ -82,7 +89,6 @@ use self::state::{ConfigBackendClient, SchedulerState}; use ballista_core::config::BallistaConfig; use ballista_core::execution_plans::ShuffleWriterExec; use ballista_core::serde::scheduler::to_proto::hash_partitioning_to_proto; -use datafusion::datasource::parquet::ParquetTableDescriptor; use datafusion::prelude::{ExecutionConfig, ExecutionContext}; use std::time::{Instant, SystemTime, UNIX_EPOCH}; @@ -272,6 +278,10 @@ impl SchedulerGrpc for SchedulerServer { &self, request: Request, ) -> std::result::Result, tonic::Status> { + // TODO support multiple object stores + let obj_store = LocalFileSystem {}; + // TODO shouldn't this take a ListingOption object as input? + let GetFileMetadataParams { path, file_type } = request.into_inner(); let file_type: FileType = file_type.try_into().map_err(|e| { @@ -280,34 +290,34 @@ impl SchedulerGrpc for SchedulerServer { tonic::Status::internal(msg) })?; - match file_type { - FileType::Parquet => { - let parquet_desc = ParquetTableDescriptor::new(&path).map_err(|e| { - let msg = format!("Error opening parquet files: {}", e); - error!("{}", msg); - tonic::Status::internal(msg) - })?; - - let partitions = parquet_desc - .descriptor - .partition_files - .iter() - .map(|pf| FilePartitionMetadata { - filename: vec![pf.path.clone()], - }) - .collect(); - - //TODO include statistics and any other info needed to reconstruct ParquetExec - Ok(Response::new(GetFileMetadataResult { - schema: Some(parquet_desc.schema().as_ref().into()), - partitions, - })) - } + let file_format: Arc = match file_type { + FileType::Parquet => Ok(Arc::new(ParquetFormat::default())), //TODO implement for CSV _ => Err(tonic::Status::unimplemented( "get_file_metadata unsupported file type", )), - } + }?; + + let file_metas = obj_store.list_file(&path).await.map_err(|e| { + let msg = format!("Error listing files: {}", e); + error!("{}", msg); + tonic::Status::internal(msg) + })?; + + let obj_readers = file_metas.map(move |f| obj_store.file_reader(f?.sized_file)); + + let schema = file_format + .infer_schema(Box::pin(obj_readers)) + .await + .map_err(|e| { + let msg = format!("Error infering schema: {}", e); + error!("{}", msg); + tonic::Status::internal(msg) + })?; + + Ok(Response::new(GetFileMetadataResult { + schema: Some(schema.as_ref().into()), + })) } async fn execute_query( @@ -343,7 +353,7 @@ impl SchedulerGrpc for SchedulerServer { //TODO we can't just create a new context because we need a context that has // tables registered from previous SQL statements that have been executed let mut ctx = create_datafusion_context(&config); - let df = ctx.sql(&sql).map_err(|e| { + let df = ctx.sql(&sql).await.map_err(|e| { let msg = format!("Error parsing SQL: {}", e); error!("{}", msg); tonic::Status::internal(msg) diff --git a/ballista/rust/scheduler/src/planner.rs b/ballista/rust/scheduler/src/planner.rs index 3d5712ae8a67..3291a62abe64 100644 --- a/ballista/rust/scheduler/src/planner.rs +++ b/ballista/rust/scheduler/src/planner.rs @@ -271,15 +271,17 @@ mod test { #[tokio::test] async fn distributed_hash_aggregate_plan() -> Result<(), BallistaError> { - let mut ctx = datafusion_test_context("testdata")?; + let mut ctx = datafusion_test_context("testdata").await?; // simplified form of TPC-H query 1 - let df = ctx.sql( - "select l_returnflag, sum(l_extendedprice * 1) as sum_disc_price + let df = ctx + .sql( + "select l_returnflag, sum(l_extendedprice * 1) as sum_disc_price from lineitem group by l_returnflag order by l_returnflag", - )?; + ) + .await?; let plan = df.to_logical_plan(); let plan = ctx.optimize(&plan)?; @@ -356,11 +358,12 @@ mod test { #[tokio::test] async fn distributed_join_plan() -> Result<(), BallistaError> { - let mut ctx = datafusion_test_context("testdata")?; + let mut ctx = datafusion_test_context("testdata").await?; // simplified form of TPC-H query 12 - let df = ctx.sql( - "select + let df = ctx + .sql( + "select l_shipmode, sum(case when o_orderpriority = '1-URGENT' @@ -391,7 +394,8 @@ group by order by l_shipmode; ", - )?; + ) + .await?; let plan = df.to_logical_plan(); let plan = ctx.optimize(&plan)?; @@ -529,15 +533,17 @@ order by #[tokio::test] async fn roundtrip_serde_hash_aggregate() -> Result<(), BallistaError> { - let mut ctx = datafusion_test_context("testdata")?; + let mut ctx = datafusion_test_context("testdata").await?; // simplified form of TPC-H query 1 - let df = ctx.sql( - "select l_returnflag, sum(l_extendedprice * 1) as sum_disc_price + let df = ctx + .sql( + "select l_returnflag, sum(l_extendedprice * 1) as sum_disc_price from lineitem group by l_returnflag order by l_returnflag", - )?; + ) + .await?; let plan = df.to_logical_plan(); let plan = ctx.optimize(&plan)?; diff --git a/ballista/rust/scheduler/src/test_utils.rs b/ballista/rust/scheduler/src/test_utils.rs index d19730998ec4..b9d7ee42f48b 100644 --- a/ballista/rust/scheduler/src/test_utils.rs +++ b/ballista/rust/scheduler/src/test_utils.rs @@ -19,13 +19,13 @@ use ballista_core::error::Result; use datafusion::arrow::datatypes::{DataType, Field, Schema}; use datafusion::execution::context::{ExecutionConfig, ExecutionContext}; -use datafusion::physical_plan::csv::CsvReadOptions; +use datafusion::prelude::CsvReadOptions; pub const TPCH_TABLES: &[&str] = &[ "part", "supplier", "partsupp", "customer", "orders", "lineitem", "nation", "region", ]; -pub fn datafusion_test_context(path: &str) -> Result { +pub async fn datafusion_test_context(path: &str) -> Result { let default_shuffle_partitions = 2; let config = ExecutionConfig::new().with_target_partitions(default_shuffle_partitions); @@ -38,7 +38,7 @@ pub fn datafusion_test_context(path: &str) -> Result { .has_header(false) .file_extension(".tbl"); let dir = format!("{}/{}", path, table); - ctx.register_csv(table, &dir, options)?; + ctx.register_csv(table, &dir, options).await?; } Ok(ctx) } diff --git a/benchmarks/src/bin/nyctaxi.rs b/benchmarks/src/bin/nyctaxi.rs index a88494fc8547..59fc69180368 100644 --- a/benchmarks/src/bin/nyctaxi.rs +++ b/benchmarks/src/bin/nyctaxi.rs @@ -29,7 +29,7 @@ use datafusion::error::Result; use datafusion::execution::context::{ExecutionConfig, ExecutionContext}; use datafusion::physical_plan::collect; -use datafusion::physical_plan::csv::CsvReadOptions; +use datafusion::prelude::CsvReadOptions; use structopt::StructOpt; #[cfg(feature = "snmalloc")] @@ -80,9 +80,9 @@ async fn main() -> Result<()> { "csv" => { let schema = nyctaxi_schema(); let options = CsvReadOptions::new().schema(&schema).has_header(true); - ctx.register_csv("tripdata", path, options)? + ctx.register_csv("tripdata", path, options).await? } - "parquet" => ctx.register_parquet("tripdata", path)?, + "parquet" => ctx.register_parquet("tripdata", path).await?, other => { println!("Invalid file format '{}'", other); process::exit(-1); diff --git a/benchmarks/src/bin/tpch.rs b/benchmarks/src/bin/tpch.rs index 203c186e1ec3..bfe87efb9f74 100644 --- a/benchmarks/src/bin/tpch.rs +++ b/benchmarks/src/bin/tpch.rs @@ -28,11 +28,7 @@ use std::{ use ballista::context::BallistaContext; use ballista::prelude::{BallistaConfig, BALLISTA_DEFAULT_SHUFFLE_PARTITIONS}; -use datafusion::arrow::datatypes::{DataType, Field, Schema}; -use datafusion::arrow::record_batch::RecordBatch; -use datafusion::arrow::util::pretty; -use datafusion::datasource::parquet::ParquetTable; -use datafusion::datasource::{CsvFile, MemTable, TableProvider}; +use datafusion::datasource::{MemTable, TableProvider}; use datafusion::error::{DataFusionError, Result}; use datafusion::logical_plan::LogicalPlan; use datafusion::parquet::basic::Compression; @@ -40,6 +36,20 @@ use datafusion::parquet::file::properties::WriterProperties; use datafusion::physical_plan::display::DisplayableExecutionPlan; use datafusion::physical_plan::{collect, displayable}; use datafusion::prelude::*; +use datafusion::{ + arrow::datatypes::{DataType, Field, Schema}, + datasource::file_format::{csv::CsvFormat, FileFormat}, +}; +use datafusion::{ + arrow::record_batch::RecordBatch, datasource::file_format::parquet::ParquetFormat, +}; +use datafusion::{ + arrow::util::pretty, + datasource::{ + listing::{ListingOptions, ListingTable}, + object_store::local::LocalFileSystem, + }, +}; use structopt::StructOpt; @@ -272,6 +282,7 @@ async fn benchmark_ballista(opt: BallistaBenchmarkOpt) -> Result<()> { .has_header(false) .file_extension(".tbl"); ctx.register_csv(table, &path, options) + .await .map_err(|e| DataFusionError::Plan(format!("{:?}", e)))?; } "csv" => { @@ -279,11 +290,13 @@ async fn benchmark_ballista(opt: BallistaBenchmarkOpt) -> Result<()> { let schema = get_schema(table); let options = CsvReadOptions::new().schema(&schema).has_header(true); ctx.register_csv(table, &path, options) + .await .map_err(|e| DataFusionError::Plan(format!("{:?}", e)))?; } "parquet" => { let path = format!("{}/{}", path, table); ctx.register_parquet(table, &path) + .await .map_err(|e| DataFusionError::Plan(format!("{:?}", e)))?; } other => { @@ -301,6 +314,7 @@ async fn benchmark_ballista(opt: BallistaBenchmarkOpt) -> Result<()> { let start = Instant::now(); let df = ctx .sql(&sql) + .await .map_err(|e| DataFusionError::Plan(format!("{:?}", e)))?; let batches = df .collect() @@ -384,7 +398,7 @@ async fn convert_tbl(opt: ConvertOpt) -> Result<()> { let mut ctx = ExecutionContext::with_config(config); // build plan to read the TBL file - let mut csv = ctx.read_csv(&input_path, options)?; + let mut csv = ctx.read_csv(&input_path, options).await?; // optionally, repartition the file if opt.partitions > 1 { @@ -445,40 +459,52 @@ fn get_table( table_format: &str, target_partitions: usize, ) -> Result> { - match table_format { - // dbgen creates .tbl ('|' delimited) files without header - "tbl" => { - let path = format!("{}/{}.tbl", path, table); - let schema = get_schema(table); - let options = CsvReadOptions::new() - .schema(&schema) - .delimiter(b'|') - .has_header(false) - .file_extension(".tbl"); + let (format, path, extension): (Arc, String, &'static str) = + match table_format { + // dbgen creates .tbl ('|' delimited) files without header + "tbl" => { + let path = format!("{}/{}.tbl", path, table); - Ok(Arc::new(CsvFile::try_new(&path, options)?)) - } - "csv" => { - let path = format!("{}/{}", path, table); - let schema = get_schema(table); - let options = CsvReadOptions::new().schema(&schema).has_header(true); + let format = CsvFormat::default() + .with_delimiter(b'|') + .with_has_header(false); - Ok(Arc::new(CsvFile::try_new(&path, options)?)) - } - "parquet" => { - let path = format!("{}/{}", path, table); - let schema = get_schema(table); - Ok(Arc::new(ParquetTable::try_new_with_schema( - &path, - schema, - target_partitions, - false, - )?)) - } - other => { - unimplemented!("Invalid file format '{}'", other); - } - } + (Arc::new(format), path, ".tbl") + } + "csv" => { + let path = format!("{}/{}", path, table); + let format = CsvFormat::default() + .with_delimiter(b',') + .with_has_header(true); + + (Arc::new(format), path, ".csv") + } + "parquet" => { + let path = format!("{}/{}", path, table); + let format = ParquetFormat::default().with_enable_pruning(true); + + (Arc::new(format), path, ".parquet") + } + other => { + unimplemented!("Invalid file format '{}'", other); + } + }; + let schema = Arc::new(get_schema(table)); + + let options = ListingOptions { + format, + file_extension: extension.to_owned(), + target_partitions, + collect_stat: true, + partitions: vec![], + }; + + Ok(Arc::new(ListingTable::new( + Arc::new(LocalFileSystem {}), + path, + schema, + options, + ))) } fn get_schema(table: &str) -> Schema { @@ -1002,7 +1028,9 @@ mod tests { .schema(&schema) .delimiter(b'|') .file_extension(".out"); - let df = ctx.read_csv(&format!("{}/answers/q{}.out", path, n), options)?; + let df = ctx + .read_csv(&format!("{}/answers/q{}.out", path, n), options) + .await?; let df = df.select( get_answer_schema(n) .fields() @@ -1081,10 +1109,13 @@ mod tests { .delimiter(b'|') .has_header(false) .file_extension(".tbl"); - let provider = CsvFile::try_new( - &format!("{}/{}.tbl", tpch_data_path, table), - options, - )?; + let listing_options = options.to_listing_options(1); + let provider = ListingTable::new( + Arc::new(LocalFileSystem {}), + format!("{}/{}.tbl", tpch_data_path, table), + Arc::new(schema), + listing_options, + ); ctx.register_table(table, Arc::new(provider))?; } @@ -1105,7 +1136,7 @@ mod tests { assert_eq!( format!("{:?}", plan), format!("{:?}", round_trip), - "opitmized logical plan round trip failed" + "optimized logical plan round trip failed" ); // test physical plan roundtrip diff --git a/datafusion-cli/src/main.rs b/datafusion-cli/src/main.rs index 4a4588812c24..481448596c23 100644 --- a/datafusion-cli/src/main.rs +++ b/datafusion-cli/src/main.rs @@ -285,8 +285,8 @@ async fn exec_and_print( let now = Instant::now(); let df = match ctx { - Context::Local(datafusion) => datafusion.sql(&sql)?, - Context::Remote(ballista) => ballista.sql(&sql)?, + Context::Local(datafusion) => datafusion.sql(&sql).await?, + Context::Remote(ballista) => ballista.sql(&sql).await?, }; let results = df.collect().await?; diff --git a/datafusion-examples/examples/avro_sql.rs b/datafusion-examples/examples/avro_sql.rs index e9676a05b1fc..f08c12bbb73a 100644 --- a/datafusion-examples/examples/avro_sql.rs +++ b/datafusion-examples/examples/avro_sql.rs @@ -18,7 +18,6 @@ use datafusion::arrow::util::pretty; use datafusion::error::Result; -use datafusion::physical_plan::avro::AvroReadOptions; use datafusion::prelude::*; /// This example demonstrates executing a simple query against an Arrow data source (Avro) and @@ -32,14 +31,17 @@ async fn main() -> Result<()> { // register avro file with the execution context let avro_file = &format!("{}/avro/alltypes_plain.avro", testdata); - ctx.register_avro("alltypes_plain", avro_file, AvroReadOptions::default())?; + ctx.register_avro("alltypes_plain", avro_file, AvroReadOptions::default()) + .await?; // execute the query - let df = ctx.sql( - "SELECT int_col, double_col, CAST(date_string_col as VARCHAR) \ + let df = ctx + .sql( + "SELECT int_col, double_col, CAST(date_string_col as VARCHAR) \ FROM alltypes_plain \ WHERE id > 1 AND tinyint_col < double_col", - )?; + ) + .await?; let results = df.collect().await?; // print the results diff --git a/datafusion-examples/examples/csv_sql.rs b/datafusion-examples/examples/csv_sql.rs index a1cdf5d6e0ab..5ad9bd7d4385 100644 --- a/datafusion-examples/examples/csv_sql.rs +++ b/datafusion-examples/examples/csv_sql.rs @@ -32,15 +32,18 @@ async fn main() -> Result<()> { "aggregate_test_100", &format!("{}/csv/aggregate_test_100.csv", testdata), CsvReadOptions::new(), - )?; + ) + .await?; // execute the query - let df = ctx.sql( - "SELECT c1, MIN(c12), MAX(c12) \ + let df = ctx + .sql( + "SELECT c1, MIN(c12), MAX(c12) \ FROM aggregate_test_100 \ WHERE c11 > 0.1 AND c11 < 0.9 \ GROUP BY c1", - )?; + ) + .await?; // print the results df.show().await?; diff --git a/datafusion-examples/examples/dataframe.rs b/datafusion-examples/examples/dataframe.rs index 013f3224ae59..6fd34610ba5c 100644 --- a/datafusion-examples/examples/dataframe.rs +++ b/datafusion-examples/examples/dataframe.rs @@ -31,7 +31,8 @@ async fn main() -> Result<()> { // define the query using the DataFrame trait let df = ctx - .read_parquet(filename)? + .read_parquet(filename) + .await? .select_columns(&["id", "bool_col", "timestamp_col"])? .filter(col("id").gt(lit(1)))?; diff --git a/datafusion-examples/examples/flight_server.rs b/datafusion-examples/examples/flight_server.rs index 138434ea2482..c26dcce59f69 100644 --- a/datafusion-examples/examples/flight_server.rs +++ b/datafusion-examples/examples/flight_server.rs @@ -16,14 +16,16 @@ // under the License. use std::pin::Pin; +use std::sync::Arc; use arrow_flight::SchemaAsIpc; +use datafusion::datasource::file_format::parquet::ParquetFormat; +use datafusion::datasource::listing::ListingOptions; +use datafusion::datasource::object_store::local::LocalFileSystem; use futures::Stream; use tonic::transport::Server; use tonic::{Request, Response, Status, Streaming}; -use datafusion::datasource::parquet::ParquetTable; -use datafusion::datasource::TableProvider; use datafusion::prelude::*; use arrow_flight::{ @@ -65,10 +67,15 @@ impl FlightService for FlightServiceImpl { ) -> Result, Status> { let request = request.into_inner(); - let table = ParquetTable::try_new(&request.path[0], num_cpus::get()).unwrap(); + let listing_options = ListingOptions::new(Arc::new(ParquetFormat::default())); + + let schema = listing_options + .infer_schema(Arc::new(LocalFileSystem {}), &request.path[0]) + .await + .unwrap(); let options = datafusion::arrow::ipc::writer::IpcWriteOptions::default(); - let schema_result = SchemaAsIpc::new(table.schema().as_ref(), &options).into(); + let schema_result = SchemaAsIpc::new(&schema, &options).into(); Ok(Response::new(schema_result)) } @@ -92,10 +99,11 @@ impl FlightService for FlightServiceImpl { "alltypes_plain", &format!("{}/alltypes_plain.parquet", testdata), ) + .await .map_err(to_tonic_err)?; // create the DataFrame - let df = ctx.sql(sql).map_err(to_tonic_err)?; + let df = ctx.sql(sql).await.map_err(to_tonic_err)?; // execute the query let results = df.collect().await.map_err(to_tonic_err)?; diff --git a/datafusion-examples/examples/parquet_sql.rs b/datafusion-examples/examples/parquet_sql.rs index 2f3ce916f4bf..e74ed39c68ce 100644 --- a/datafusion-examples/examples/parquet_sql.rs +++ b/datafusion-examples/examples/parquet_sql.rs @@ -31,14 +31,17 @@ async fn main() -> Result<()> { ctx.register_parquet( "alltypes_plain", &format!("{}/alltypes_plain.parquet", testdata), - )?; + ) + .await?; // execute the query - let df = ctx.sql( - "SELECT int_col, double_col, CAST(date_string_col as VARCHAR) \ + let df = ctx + .sql( + "SELECT int_col, double_col, CAST(date_string_col as VARCHAR) \ FROM alltypes_plain \ WHERE id > 1 AND tinyint_col < double_col", - )?; + ) + .await?; // print the results df.show().await?; diff --git a/datafusion/benches/aggregate_query_sql.rs b/datafusion/benches/aggregate_query_sql.rs index b8fe06fd9145..dc40c61db41d 100644 --- a/datafusion/benches/aggregate_query_sql.rs +++ b/datafusion/benches/aggregate_query_sql.rs @@ -30,7 +30,7 @@ use tokio::runtime::Runtime; fn query(ctx: Arc>, sql: &str) { let rt = Runtime::new().unwrap(); - let df = ctx.lock().unwrap().sql(sql).unwrap(); + let df = rt.block_on(ctx.lock().unwrap().sql(sql)).unwrap(); criterion::black_box(rt.block_on(df.collect()).unwrap()); } diff --git a/datafusion/benches/filter_query_sql.rs b/datafusion/benches/filter_query_sql.rs index aac7f9624872..c64c52126b0d 100644 --- a/datafusion/benches/filter_query_sql.rs +++ b/datafusion/benches/filter_query_sql.rs @@ -31,7 +31,7 @@ async fn query(ctx: &mut ExecutionContext, sql: &str) { let rt = Runtime::new().unwrap(); // execute the query - let df = ctx.sql(sql).unwrap(); + let df = rt.block_on(ctx.sql(sql)).unwrap(); criterion::black_box(rt.block_on(df.collect()).unwrap()); } diff --git a/datafusion/benches/math_query_sql.rs b/datafusion/benches/math_query_sql.rs index 51e52e8acddb..4f738890460f 100644 --- a/datafusion/benches/math_query_sql.rs +++ b/datafusion/benches/math_query_sql.rs @@ -40,7 +40,7 @@ fn query(ctx: Arc>, sql: &str) { let rt = Runtime::new().unwrap(); // execute the query - let df = ctx.lock().unwrap().sql(sql).unwrap(); + let df = rt.block_on(ctx.lock().unwrap().sql(sql)).unwrap(); rt.block_on(df.collect()).unwrap(); } diff --git a/datafusion/benches/sort_limit_query_sql.rs b/datafusion/benches/sort_limit_query_sql.rs index 195bd5cf15c2..f3151d2d7140 100644 --- a/datafusion/benches/sort_limit_query_sql.rs +++ b/datafusion/benches/sort_limit_query_sql.rs @@ -18,6 +18,9 @@ #[macro_use] extern crate criterion; use criterion::Criterion; +use datafusion::datasource::file_format::csv::CsvFormat; +use datafusion::datasource::listing::{ListingOptions, ListingTable}; +use datafusion::datasource::object_store::local::LocalFileSystem; use std::sync::{Arc, Mutex}; @@ -26,7 +29,7 @@ extern crate datafusion; use arrow::datatypes::{DataType, Field, Schema}; -use datafusion::datasource::{CsvFile, CsvReadOptions, MemTable}; +use datafusion::datasource::MemTable; use datafusion::execution::context::ExecutionContext; use tokio::runtime::Runtime; @@ -35,7 +38,7 @@ fn query(ctx: Arc>, sql: &str) { let rt = Runtime::new().unwrap(); // execute the query - let df = ctx.lock().unwrap().sql(sql).unwrap(); + let df = rt.block_on(ctx.lock().unwrap().sql(sql)).unwrap(); rt.block_on(df.collect()).unwrap(); } @@ -60,11 +63,13 @@ fn create_context() -> Arc> { let testdata = datafusion::test_util::arrow_test_data(); // create CSV data source - let csv = CsvFile::try_new( - &format!("{}/csv/aggregate_test_100.csv", testdata), - CsvReadOptions::new().schema(&schema), - ) - .unwrap(); + let listing_options = ListingOptions::new(Arc::new(CsvFormat::default())); + let csv = ListingTable::new( + Arc::new(LocalFileSystem {}), + format!("{}/csv/aggregate_test_100.csv", testdata), + schema, + listing_options, + ); let rt = Runtime::new().unwrap(); diff --git a/datafusion/benches/window_query_sql.rs b/datafusion/benches/window_query_sql.rs index 7c323be2b5ed..bca4a38360fe 100644 --- a/datafusion/benches/window_query_sql.rs +++ b/datafusion/benches/window_query_sql.rs @@ -30,7 +30,7 @@ use tokio::runtime::Runtime; fn query(ctx: Arc>, sql: &str) { let rt = Runtime::new().unwrap(); - let df = ctx.lock().unwrap().sql(sql).unwrap(); + let df = rt.block_on(ctx.lock().unwrap().sql(sql)).unwrap(); criterion::black_box(rt.block_on(df.collect()).unwrap()); } diff --git a/datafusion/src/dataframe.rs b/datafusion/src/dataframe.rs index 5b157d0ac149..4bfd7206eb3e 100644 --- a/datafusion/src/dataframe.rs +++ b/datafusion/src/dataframe.rs @@ -41,9 +41,10 @@ use async_trait::async_trait; /// ``` /// # use datafusion::prelude::*; /// # use datafusion::error::Result; -/// # fn main() -> Result<()> { +/// # #[tokio::main] +/// # async fn main() -> Result<()> { /// let mut ctx = ExecutionContext::new(); -/// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new())?; +/// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new()).await?; /// let df = df.filter(col("a").lt_eq(col("b")))? /// .aggregate(vec![col("a")], vec![min(col("b"))])? /// .limit(100)?; @@ -59,9 +60,10 @@ pub trait DataFrame: Send + Sync { /// ``` /// # use datafusion::prelude::*; /// # use datafusion::error::Result; - /// # fn main() -> Result<()> { + /// # #[tokio::main] + /// # async fn main() -> Result<()> { /// let mut ctx = ExecutionContext::new(); - /// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new())?; + /// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new()).await?; /// let df = df.select_columns(&["a", "b"])?; /// # Ok(()) /// # } @@ -73,9 +75,10 @@ pub trait DataFrame: Send + Sync { /// ``` /// # use datafusion::prelude::*; /// # use datafusion::error::Result; - /// # fn main() -> Result<()> { + /// # #[tokio::main] + /// # async fn main() -> Result<()> { /// let mut ctx = ExecutionContext::new(); - /// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new())?; + /// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new()).await?; /// let df = df.select(vec![col("a") * col("b"), col("c")])?; /// # Ok(()) /// # } @@ -87,9 +90,10 @@ pub trait DataFrame: Send + Sync { /// ``` /// # use datafusion::prelude::*; /// # use datafusion::error::Result; - /// # fn main() -> Result<()> { + /// # #[tokio::main] + /// # async fn main() -> Result<()> { /// let mut ctx = ExecutionContext::new(); - /// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new())?; + /// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new()).await?; /// let df = df.filter(col("a").lt_eq(col("b")))?; /// # Ok(()) /// # } @@ -101,9 +105,10 @@ pub trait DataFrame: Send + Sync { /// ``` /// # use datafusion::prelude::*; /// # use datafusion::error::Result; - /// # fn main() -> Result<()> { + /// # #[tokio::main] + /// # async fn main() -> Result<()> { /// let mut ctx = ExecutionContext::new(); - /// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new())?; + /// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new()).await?; /// /// // The following use is the equivalent of "SELECT MIN(b) GROUP BY a" /// let _ = df.aggregate(vec![col("a")], vec![min(col("b"))])?; @@ -124,9 +129,10 @@ pub trait DataFrame: Send + Sync { /// ``` /// # use datafusion::prelude::*; /// # use datafusion::error::Result; - /// # fn main() -> Result<()> { + /// # #[tokio::main] + /// # async fn main() -> Result<()> { /// let mut ctx = ExecutionContext::new(); - /// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new())?; + /// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new()).await?; /// let df = df.limit(100)?; /// # Ok(()) /// # } @@ -138,9 +144,10 @@ pub trait DataFrame: Send + Sync { /// ``` /// # use datafusion::prelude::*; /// # use datafusion::error::Result; - /// # fn main() -> Result<()> { + /// # #[tokio::main] + /// # async fn main() -> Result<()> { /// let mut ctx = ExecutionContext::new(); - /// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new())?; + /// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new()).await?; /// let df = df.union(df.clone())?; /// # Ok(()) /// # } @@ -152,9 +159,10 @@ pub trait DataFrame: Send + Sync { /// ``` /// # use datafusion::prelude::*; /// # use datafusion::error::Result; - /// # fn main() -> Result<()> { + /// # #[tokio::main] + /// # async fn main() -> Result<()> { /// let mut ctx = ExecutionContext::new(); - /// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new())?; + /// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new()).await?; /// let df = df.union(df.clone())?; /// let df = df.distinct()?; /// # Ok(()) @@ -168,9 +176,10 @@ pub trait DataFrame: Send + Sync { /// ``` /// # use datafusion::prelude::*; /// # use datafusion::error::Result; - /// # fn main() -> Result<()> { + /// # #[tokio::main] + /// # async fn main() -> Result<()> { /// let mut ctx = ExecutionContext::new(); - /// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new())?; + /// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new()).await?; /// let df = df.sort(vec![col("a").sort(true, true), col("b").sort(false, false)])?; /// # Ok(()) /// # } @@ -185,8 +194,8 @@ pub trait DataFrame: Send + Sync { /// # #[tokio::main] /// # async fn main() -> Result<()> { /// let mut ctx = ExecutionContext::new(); - /// let left = ctx.read_csv("tests/example.csv", CsvReadOptions::new())?; - /// let right = ctx.read_csv("tests/example.csv", CsvReadOptions::new())? + /// let left = ctx.read_csv("tests/example.csv", CsvReadOptions::new()).await?; + /// let right = ctx.read_csv("tests/example.csv", CsvReadOptions::new()).await? /// .select(vec![ /// col("a").alias("a2"), /// col("b").alias("b2"), @@ -211,9 +220,10 @@ pub trait DataFrame: Send + Sync { /// ``` /// # use datafusion::prelude::*; /// # use datafusion::error::Result; - /// # fn main() -> Result<()> { + /// # #[tokio::main] + /// # async fn main() -> Result<()> { /// let mut ctx = ExecutionContext::new(); - /// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new())?; + /// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new()).await?; /// let df1 = df.repartition(Partitioning::RoundRobinBatch(4))?; /// # Ok(()) /// # } @@ -231,7 +241,7 @@ pub trait DataFrame: Send + Sync { /// # #[tokio::main] /// # async fn main() -> Result<()> { /// let mut ctx = ExecutionContext::new(); - /// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new())?; + /// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new()).await?; /// let batches = df.collect().await?; /// # Ok(()) /// # } @@ -246,7 +256,7 @@ pub trait DataFrame: Send + Sync { /// # #[tokio::main] /// # async fn main() -> Result<()> { /// let mut ctx = ExecutionContext::new(); - /// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new())?; + /// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new()).await?; /// df.show().await?; /// # Ok(()) /// # } @@ -261,7 +271,7 @@ pub trait DataFrame: Send + Sync { /// # #[tokio::main] /// # async fn main() -> Result<()> { /// let mut ctx = ExecutionContext::new(); - /// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new())?; + /// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new()).await?; /// df.show_limit(10).await?; /// # Ok(()) /// # } @@ -276,7 +286,7 @@ pub trait DataFrame: Send + Sync { /// # #[tokio::main] /// # async fn main() -> Result<()> { /// let mut ctx = ExecutionContext::new(); - /// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new())?; + /// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new()).await?; /// let stream = df.execute_stream().await?; /// # Ok(()) /// # } @@ -292,7 +302,7 @@ pub trait DataFrame: Send + Sync { /// # #[tokio::main] /// # async fn main() -> Result<()> { /// let mut ctx = ExecutionContext::new(); - /// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new())?; + /// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new()).await?; /// let batches = df.collect_partitioned().await?; /// # Ok(()) /// # } @@ -307,7 +317,7 @@ pub trait DataFrame: Send + Sync { /// # #[tokio::main] /// # async fn main() -> Result<()> { /// let mut ctx = ExecutionContext::new(); - /// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new())?; + /// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new()).await?; /// let batches = df.execute_stream_partitioned().await?; /// # Ok(()) /// # } @@ -320,9 +330,10 @@ pub trait DataFrame: Send + Sync { /// ``` /// # use datafusion::prelude::*; /// # use datafusion::error::Result; - /// # fn main() -> Result<()> { + /// # #[tokio::main] + /// # async fn main() -> Result<()> { /// let mut ctx = ExecutionContext::new(); - /// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new())?; + /// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new()).await?; /// let schema = df.schema(); /// # Ok(()) /// # } @@ -342,7 +353,7 @@ pub trait DataFrame: Send + Sync { /// # #[tokio::main] /// # async fn main() -> Result<()> { /// let mut ctx = ExecutionContext::new(); - /// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new())?; + /// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new()).await?; /// let batches = df.limit(100)?.explain(false, false)?.collect().await?; /// # Ok(()) /// # } @@ -354,9 +365,10 @@ pub trait DataFrame: Send + Sync { /// ``` /// # use datafusion::prelude::*; /// # use datafusion::error::Result; - /// # fn main() -> Result<()> { + /// # #[tokio::main] + /// # async fn main() -> Result<()> { /// let mut ctx = ExecutionContext::new(); - /// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new())?; + /// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new()).await?; /// let f = df.registry(); /// // use f.udf("name", vec![...]) to use the udf /// # Ok(()) diff --git a/datafusion/src/datasource/avro.rs b/datafusion/src/datasource/avro.rs deleted file mode 100644 index ee5cea51d991..000000000000 --- a/datafusion/src/datasource/avro.rs +++ /dev/null @@ -1,426 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -//! Line-delimited Avro data source -//! -//! This data source allows Line-delimited Avro records or files to be used as input for queries. -//! - -use std::{ - any::Any, - io::{Read, Seek}, - sync::{Arc, Mutex}, -}; - -use arrow::datatypes::SchemaRef; -use async_trait::async_trait; - -use crate::physical_plan::avro::{AvroExec, AvroReadOptions}; -use crate::{ - datasource::{Source, TableProvider}, - error::{DataFusionError, Result}, - physical_plan::{common, ExecutionPlan}, -}; - -trait SeekRead: Read + Seek {} - -impl SeekRead for T {} - -/// Represents a line-delimited Avro file with a provided schema -pub struct AvroFile { - source: Source>, - schema: SchemaRef, - file_extension: String, -} - -impl AvroFile { - /// Attempt to initialize a `AvroFile` from a path. The schema can be read automatically. - pub fn try_new(path: &str, options: AvroReadOptions) -> Result { - let schema = if let Some(schema) = options.schema { - schema - } else { - let filenames = - common::build_checked_file_list(path, options.file_extension)?; - Arc::new(AvroExec::try_read_schema(&filenames)?) - }; - - Ok(Self { - source: Source::Path(path.to_string()), - schema, - file_extension: options.file_extension.to_string(), - }) - } - - /// Attempt to initialize a `AvroFile` from a reader. The schema MUST be provided in options - pub fn try_new_from_reader( - reader: R, - options: AvroReadOptions, - ) -> Result { - let schema = match options.schema { - Some(s) => s, - None => { - return Err(DataFusionError::Execution( - "Schema must be provided to CsvRead".to_string(), - )); - } - }; - Ok(Self { - source: Source::Reader(Mutex::new(Some(Box::new(reader)))), - schema, - file_extension: String::new(), - }) - } - - /// Attempt to initialize an AvroFile from a reader impls Seek. The schema can be read automatically. - pub fn try_new_from_reader_schema( - mut reader: R, - options: AvroReadOptions, - ) -> Result { - let schema = { - if let Some(schema) = options.schema { - schema - } else { - Arc::new(crate::avro_to_arrow::read_avro_schema_from_reader( - &mut reader, - )?) - } - }; - - Ok(Self { - source: Source::Reader(Mutex::new(Some(Box::new(reader)))), - schema, - file_extension: String::new(), - }) - } - - /// Get the path for Avro file(s) represented by this AvroFile instance - pub fn path(&self) -> &str { - match &self.source { - Source::Reader(_) => "", - Source::Path(path) => path, - } - } - - /// Get the file extension for the Avro file(s) represented by this AvroFile instance - pub fn file_extension(&self) -> &str { - &self.file_extension - } -} - -#[async_trait] -impl TableProvider for AvroFile { - fn as_any(&self) -> &dyn Any { - self - } - - fn schema(&self) -> SchemaRef { - self.schema.clone() - } - - async fn scan( - &self, - projection: &Option>, - batch_size: usize, - _filters: &[crate::logical_plan::Expr], - limit: Option, - ) -> Result> { - let opts = AvroReadOptions { - schema: Some(self.schema.clone()), - file_extension: self.file_extension.as_str(), - }; - let batch_size = limit - .map(|l| std::cmp::min(l, batch_size)) - .unwrap_or(batch_size); - - let exec = match &self.source { - Source::Reader(maybe_reader) => { - if let Some(rdr) = maybe_reader.lock().unwrap().take() { - AvroExec::try_new_from_reader( - rdr, - opts, - projection.clone(), - batch_size, - limit, - )? - } else { - return Err(DataFusionError::Execution( - "You can only read once if the data comes from a reader" - .to_string(), - )); - } - } - Source::Path(p) => { - AvroExec::try_from_path(p, opts, projection.clone(), batch_size, limit)? - } - }; - Ok(Arc::new(exec)) - } -} - -#[cfg(test)] -#[cfg(feature = "avro")] -mod tests { - use arrow::array::{ - BinaryArray, BooleanArray, Float32Array, Float64Array, Int32Array, - TimestampMicrosecondArray, - }; - use arrow::record_batch::RecordBatch; - use futures::StreamExt; - - use super::*; - - #[tokio::test] - async fn read_small_batches() -> Result<()> { - let table = load_table("alltypes_plain.avro")?; - let projection = None; - let exec = table.scan(&projection, 2, &[], None).await?; - let stream = exec.execute(0).await?; - - let _ = stream - .map(|batch| { - let batch = batch.unwrap(); - assert_eq!(11, batch.num_columns()); - assert_eq!(2, batch.num_rows()); - }) - .fold(0, |acc, _| async move { acc + 1i32 }) - .await; - - Ok(()) - } - - #[cfg(feature = "avro")] - #[tokio::test] - async fn read_alltypes_plain_avro() -> Result<()> { - let table = load_table("alltypes_plain.avro")?; - - let x: Vec = table - .schema() - .fields() - .iter() - .map(|f| format!("{}: {:?}", f.name(), f.data_type())) - .collect(); - let y = x.join("\n"); - assert_eq!( - "id: Int32\n\ - bool_col: Boolean\n\ - tinyint_col: Int32\n\ - smallint_col: Int32\n\ - int_col: Int32\n\ - bigint_col: Int64\n\ - float_col: Float32\n\ - double_col: Float64\n\ - date_string_col: Binary\n\ - string_col: Binary\n\ - timestamp_col: Timestamp(Microsecond, None)", - y - ); - - let projection = None; - let batch = get_first_batch(table, &projection).await?; - let expected = vec![ - "+----+----------+-------------+--------------+---------+------------+-----------+------------+------------------+------------+---------------------+", - "| id | bool_col | tinyint_col | smallint_col | int_col | bigint_col | float_col | double_col | date_string_col | string_col | timestamp_col |", - "+----+----------+-------------+--------------+---------+------------+-----------+------------+------------------+------------+---------------------+", - "| 4 | true | 0 | 0 | 0 | 0 | 0 | 0 | 30332f30312f3039 | 30 | 2009-03-01 00:00:00 |", - "| 5 | false | 1 | 1 | 1 | 10 | 1.1 | 10.1 | 30332f30312f3039 | 31 | 2009-03-01 00:01:00 |", - "| 6 | true | 0 | 0 | 0 | 0 | 0 | 0 | 30342f30312f3039 | 30 | 2009-04-01 00:00:00 |", - "| 7 | false | 1 | 1 | 1 | 10 | 1.1 | 10.1 | 30342f30312f3039 | 31 | 2009-04-01 00:01:00 |", - "| 2 | true | 0 | 0 | 0 | 0 | 0 | 0 | 30322f30312f3039 | 30 | 2009-02-01 00:00:00 |", - "| 3 | false | 1 | 1 | 1 | 10 | 1.1 | 10.1 | 30322f30312f3039 | 31 | 2009-02-01 00:01:00 |", - "| 0 | true | 0 | 0 | 0 | 0 | 0 | 0 | 30312f30312f3039 | 30 | 2009-01-01 00:00:00 |", - "| 1 | false | 1 | 1 | 1 | 10 | 1.1 | 10.1 | 30312f30312f3039 | 31 | 2009-01-01 00:01:00 |", - "+----+----------+-------------+--------------+---------+------------+-----------+------------+------------------+------------+---------------------+", - ]; - - crate::assert_batches_eq!(expected, &[batch]); - Ok(()) - } - - #[tokio::test] - async fn read_bool_alltypes_plain_avro() -> Result<()> { - let table = load_table("alltypes_plain.avro")?; - let projection = Some(vec![1]); - let batch = get_first_batch(table, &projection).await?; - - assert_eq!(1, batch.num_columns()); - assert_eq!(8, batch.num_rows()); - - let array = batch - .column(0) - .as_any() - .downcast_ref::() - .unwrap(); - let mut values: Vec = vec![]; - for i in 0..batch.num_rows() { - values.push(array.value(i)); - } - - assert_eq!( - "[true, false, true, false, true, false, true, false]", - format!("{:?}", values) - ); - - Ok(()) - } - - #[tokio::test] - async fn read_i32_alltypes_plain_avro() -> Result<()> { - let table = load_table("alltypes_plain.avro")?; - let projection = Some(vec![0]); - let batch = get_first_batch(table, &projection).await?; - - assert_eq!(1, batch.num_columns()); - assert_eq!(8, batch.num_rows()); - - let array = batch - .column(0) - .as_any() - .downcast_ref::() - .unwrap(); - let mut values: Vec = vec![]; - for i in 0..batch.num_rows() { - values.push(array.value(i)); - } - - assert_eq!("[4, 5, 6, 7, 2, 3, 0, 1]", format!("{:?}", values)); - - Ok(()) - } - - #[tokio::test] - async fn read_i96_alltypes_plain_avro() -> Result<()> { - let table = load_table("alltypes_plain.avro")?; - let projection = Some(vec![10]); - let batch = get_first_batch(table, &projection).await?; - assert_eq!(1, batch.num_columns()); - assert_eq!(8, batch.num_rows()); - - let array = batch - .column(0) - .as_any() - .downcast_ref::() - .unwrap(); - let mut values: Vec = vec![]; - for i in 0..batch.num_rows() { - values.push(array.value(i)); - } - - assert_eq!("[1235865600000000, 1235865660000000, 1238544000000000, 1238544060000000, 1233446400000000, 1233446460000000, 1230768000000000, 1230768060000000]", format!("{:?}", values)); - - Ok(()) - } - - #[tokio::test] - async fn read_f32_alltypes_plain_avro() -> Result<()> { - let table = load_table("alltypes_plain.avro")?; - let projection = Some(vec![6]); - let batch = get_first_batch(table, &projection).await?; - - assert_eq!(1, batch.num_columns()); - assert_eq!(8, batch.num_rows()); - - let array = batch - .column(0) - .as_any() - .downcast_ref::() - .unwrap(); - let mut values: Vec = vec![]; - for i in 0..batch.num_rows() { - values.push(array.value(i)); - } - - assert_eq!( - "[0.0, 1.1, 0.0, 1.1, 0.0, 1.1, 0.0, 1.1]", - format!("{:?}", values) - ); - - Ok(()) - } - - #[tokio::test] - async fn read_f64_alltypes_plain_avro() -> Result<()> { - let table = load_table("alltypes_plain.avro")?; - let projection = Some(vec![7]); - let batch = get_first_batch(table, &projection).await?; - - assert_eq!(1, batch.num_columns()); - assert_eq!(8, batch.num_rows()); - - let array = batch - .column(0) - .as_any() - .downcast_ref::() - .unwrap(); - let mut values: Vec = vec![]; - for i in 0..batch.num_rows() { - values.push(array.value(i)); - } - - assert_eq!( - "[0.0, 10.1, 0.0, 10.1, 0.0, 10.1, 0.0, 10.1]", - format!("{:?}", values) - ); - - Ok(()) - } - - #[tokio::test] - async fn read_binary_alltypes_plain_avro() -> Result<()> { - let table = load_table("alltypes_plain.avro")?; - let projection = Some(vec![9]); - let batch = get_first_batch(table, &projection).await?; - - assert_eq!(1, batch.num_columns()); - assert_eq!(8, batch.num_rows()); - - let array = batch - .column(0) - .as_any() - .downcast_ref::() - .unwrap(); - let mut values: Vec<&str> = vec![]; - for i in 0..batch.num_rows() { - values.push(std::str::from_utf8(array.value(i)).unwrap()); - } - - assert_eq!( - "[\"0\", \"1\", \"0\", \"1\", \"0\", \"1\", \"0\", \"1\"]", - format!("{:?}", values) - ); - - Ok(()) - } - - fn load_table(name: &str) -> Result> { - let testdata = crate::test_util::arrow_test_data(); - let filename = format!("{}/avro/{}", testdata, name); - let table = AvroFile::try_new(&filename, AvroReadOptions::default())?; - Ok(Arc::new(table)) - } - - async fn get_first_batch( - table: Arc, - projection: &Option>, - ) -> Result { - let exec = table.scan(projection, 1024, &[], None).await?; - let mut it = exec.execute(0).await?; - it.next() - .await - .expect("should have received at least one batch") - .map_err(|e| e.into()) - } -} diff --git a/datafusion/src/datasource/csv.rs b/datafusion/src/datasource/csv.rs deleted file mode 100644 index d47312e8b745..000000000000 --- a/datafusion/src/datasource/csv.rs +++ /dev/null @@ -1,245 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -//! CSV data source -//! -//! This CSV data source allows CSV files to be used as input for queries. -//! -//! Example: -//! -//! ``` -//! use datafusion::datasource::TableProvider; -//! use datafusion::datasource::csv::{CsvFile, CsvReadOptions}; -//! -//! let testdata = datafusion::test_util::arrow_test_data(); -//! let csvdata = CsvFile::try_new( -//! &format!("{}/csv/aggregate_test_100.csv", testdata), -//! CsvReadOptions::new().delimiter(b'|'), -//! ).unwrap(); -//! let schema = csvdata.schema(); -//! ``` - -use arrow::datatypes::SchemaRef; -use async_trait::async_trait; -use std::any::Any; -use std::io::{Read, Seek}; -use std::string::String; -use std::sync::{Arc, Mutex}; - -use crate::datasource::{Source, TableProvider}; -use crate::error::{DataFusionError, Result}; -use crate::logical_plan::Expr; -use crate::physical_plan::csv::CsvExec; -pub use crate::physical_plan::csv::CsvReadOptions; -use crate::physical_plan::{common, ExecutionPlan}; - -/// Represents a CSV file with a provided schema -pub struct CsvFile { - source: Source, - schema: SchemaRef, - has_header: bool, - delimiter: u8, - file_extension: String, -} - -impl CsvFile { - /// Attempt to initialize a new `CsvFile` from a file path - pub fn try_new(path: impl Into, options: CsvReadOptions) -> Result { - let path = path.into(); - let schema = Arc::new(match options.schema { - Some(s) => s.clone(), - None => { - let filenames = common::build_file_list(&path, options.file_extension)?; - if filenames.is_empty() { - return Err(DataFusionError::Plan(format!( - "No files found at {path} with file extension {file_extension}", - path = path, - file_extension = options.file_extension - ))); - } - CsvExec::try_infer_schema(&filenames, &options)? - } - }); - - Ok(Self { - source: Source::Path(path), - schema, - has_header: options.has_header, - delimiter: options.delimiter, - file_extension: String::from(options.file_extension), - }) - } - - /// Attempt to initialize a `CsvFile` from a reader. The schema MUST be provided in options. - pub fn try_new_from_reader( - reader: R, - options: CsvReadOptions, - ) -> Result { - let schema = Arc::new(match options.schema { - Some(s) => s.clone(), - None => { - return Err(DataFusionError::Execution( - "Schema must be provided to CsvRead".to_string(), - )); - } - }); - - Ok(Self { - source: Source::Reader(Mutex::new(Some(Box::new(reader)))), - schema, - has_header: options.has_header, - delimiter: options.delimiter, - file_extension: String::new(), - }) - } - - /// Attempt to initialize a `CsvRead` from a reader impls `Seek`. The schema can be inferred automatically. - pub fn try_new_from_reader_infer_schema( - mut reader: R, - options: CsvReadOptions, - ) -> Result { - let schema = Arc::new(match options.schema { - Some(s) => s.clone(), - None => { - let (schema, _) = arrow::csv::reader::infer_file_schema( - &mut reader, - options.delimiter, - Some(options.schema_infer_max_records), - options.has_header, - )?; - schema - } - }); - - Ok(Self { - source: Source::Reader(Mutex::new(Some(Box::new(reader)))), - schema, - has_header: options.has_header, - delimiter: options.delimiter, - file_extension: String::new(), - }) - } - - /// Get the path for the CSV file(s) represented by this CsvFile instance - pub fn path(&self) -> &str { - match &self.source { - Source::Reader(_) => "", - Source::Path(path) => path, - } - } - - /// Determine whether the CSV file(s) represented by this CsvFile instance have a header row - pub fn has_header(&self) -> bool { - self.has_header - } - - /// Get the delimiter for the CSV file(s) represented by this CsvFile instance - pub fn delimiter(&self) -> u8 { - self.delimiter - } - - /// Get the file extension for the CSV file(s) represented by this CsvFile instance - pub fn file_extension(&self) -> &str { - &self.file_extension - } -} - -#[async_trait] -impl TableProvider for CsvFile { - fn as_any(&self) -> &dyn Any { - self - } - - fn schema(&self) -> SchemaRef { - self.schema.clone() - } - - async fn scan( - &self, - projection: &Option>, - batch_size: usize, - _filters: &[Expr], - limit: Option, - ) -> Result> { - let opts = CsvReadOptions::new() - .schema(&self.schema) - .has_header(self.has_header) - .delimiter(self.delimiter) - .file_extension(self.file_extension.as_str()); - let batch_size = limit - .map(|l| std::cmp::min(l, batch_size)) - .unwrap_or(batch_size); - - let exec = match &self.source { - Source::Reader(maybe_reader) => { - if let Some(rdr) = maybe_reader.lock().unwrap().take() { - CsvExec::try_new_from_reader( - rdr, - opts, - projection.clone(), - batch_size, - limit, - )? - } else { - return Err(DataFusionError::Execution( - "You can only read once if the data comes from a reader" - .to_string(), - )); - } - } - Source::Path(p) => { - CsvExec::try_new(p, opts, projection.clone(), batch_size, limit)? - } - }; - Ok(Arc::new(exec)) - } -} - -#[cfg(test)] -mod tests { - use super::*; - use crate::prelude::*; - - #[tokio::test] - async fn csv_file_from_reader() -> Result<()> { - let testdata = crate::test_util::arrow_test_data(); - let filename = "aggregate_test_100.csv"; - let path = format!("{}/csv/{}", testdata, filename); - let buf = std::fs::read(path).unwrap(); - let rdr = std::io::Cursor::new(buf); - let mut ctx = ExecutionContext::new(); - ctx.register_table( - "aggregate_test", - Arc::new(CsvFile::try_new_from_reader_infer_schema( - rdr, - CsvReadOptions::new(), - )?), - )?; - let df = ctx.sql("select max(c2) from aggregate_test")?; - let batches = df.collect().await?; - assert_eq!( - batches[0] - .column(0) - .as_any() - .downcast_ref::() - .unwrap() - .value(0), - 5 - ); - Ok(()) - } -} diff --git a/datafusion/src/datasource/file_format/avro.rs b/datafusion/src/datasource/file_format/avro.rs index cb0f21640f51..7728747b8015 100644 --- a/datafusion/src/datasource/file_format/avro.rs +++ b/datafusion/src/datasource/file_format/avro.rs @@ -17,6 +17,7 @@ //! Apache Avro format abstractions +use std::any::Any; use std::sync::Arc; use arrow::datatypes::Schema; @@ -33,11 +34,15 @@ use crate::physical_plan::ExecutionPlan; use crate::physical_plan::Statistics; /// Avro `FileFormat` implementation. -#[derive(Default)] +#[derive(Default, Debug)] pub struct AvroFormat; #[async_trait] impl FileFormat for AvroFormat { + fn as_any(&self) -> &dyn Any { + self + } + async fn infer_schema(&self, mut readers: ObjectReaderStream) -> Result { let mut schemas = vec![]; while let Some(obj_reader) = readers.next().await { @@ -60,7 +65,7 @@ impl FileFormat for AvroFormat { let exec = AvroExec::new( conf.object_store, // flattening this for now because CsvExec does not support partitioning yet - conf.files.into_iter().flatten().collect(), + conf.files.into_iter().flatten().collect::>(), conf.statistics, conf.schema, conf.projection, @@ -76,11 +81,11 @@ impl FileFormat for AvroFormat { mod tests { use crate::{ datasource::{ - file_format::PartitionedFile, object_store::local::{ local_file_meta, local_object_reader, local_object_reader_stream, LocalFileSystem, }, + PartitionedFile, }, physical_plan::collect, }; diff --git a/datafusion/src/datasource/file_format/csv.rs b/datafusion/src/datasource/file_format/csv.rs index c4a6462b9bd4..4d75c65eddbb 100644 --- a/datafusion/src/datasource/file_format/csv.rs +++ b/datafusion/src/datasource/file_format/csv.rs @@ -17,6 +17,7 @@ //! CSV format abstractions +use std::any::Any; use std::sync::Arc; use arrow::datatypes::Schema; @@ -32,6 +33,7 @@ use crate::physical_plan::ExecutionPlan; use crate::physical_plan::Statistics; /// Character Separated Value `FileFormat` implementation. +#[derive(Debug)] pub struct CsvFormat { has_header: bool, delimiter: u8, @@ -63,16 +65,30 @@ impl CsvFormat { self } + /// True if the first line is a header. + pub fn has_header(&self) -> bool { + self.has_header + } + /// The character separating values within a row. /// - default to ',' pub fn with_delimiter(mut self, delimiter: u8) -> Self { self.delimiter = delimiter; self } + + /// The delimiter character. + pub fn delimiter(&self) -> u8 { + self.delimiter + } } #[async_trait] impl FileFormat for CsvFormat { + fn as_any(&self) -> &dyn Any { + self + } + async fn infer_schema(&self, mut readers: ObjectReaderStream) -> Result { let mut schemas = vec![]; @@ -111,7 +127,7 @@ impl FileFormat for CsvFormat { let exec = CsvExec::new( conf.object_store, // flattening this for now because CsvExec does not support partitioning yet - conf.files.into_iter().flatten().collect(), + conf.files.into_iter().flatten().collect::>(), conf.statistics, conf.schema, self.has_header, @@ -131,11 +147,12 @@ mod tests { use super::*; use crate::{ datasource::{ - file_format::{PartitionedFile, PhysicalPlanConfig}, + file_format::PhysicalPlanConfig, object_store::local::{ local_file_meta, local_object_reader, local_object_reader_stream, LocalFileSystem, }, + PartitionedFile, }, physical_plan::collect, }; diff --git a/datafusion/src/datasource/file_format/json.rs b/datafusion/src/datasource/file_format/json.rs index 2fa683437869..2741da31b921 100644 --- a/datafusion/src/datasource/file_format/json.rs +++ b/datafusion/src/datasource/file_format/json.rs @@ -17,6 +17,7 @@ //! Line delimited JSON format abstractions +use std::any::Any; use std::io::BufReader; use std::sync::Arc; @@ -36,6 +37,7 @@ use crate::physical_plan::ExecutionPlan; use crate::physical_plan::Statistics; /// New line delimited JSON `FileFormat` implementation. +#[derive(Debug)] pub struct JsonFormat { schema_infer_max_rec: Option, } @@ -59,6 +61,10 @@ impl JsonFormat { #[async_trait] impl FileFormat for JsonFormat { + fn as_any(&self) -> &dyn Any { + self + } + 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); @@ -91,7 +97,7 @@ impl FileFormat for JsonFormat { let exec = NdJsonExec::new( conf.object_store, // flattening this for now because NdJsonExec does not support partitioning yet - conf.files.into_iter().flatten().collect(), + conf.files.into_iter().flatten().collect::>(), conf.statistics, conf.schema, conf.projection, @@ -109,11 +115,12 @@ mod tests { use super::*; use crate::{ datasource::{ - file_format::{PartitionedFile, PhysicalPlanConfig}, + file_format::PhysicalPlanConfig, object_store::local::{ local_file_meta, local_object_reader, local_object_reader_stream, LocalFileSystem, }, + PartitionedFile, }, physical_plan::collect, }; diff --git a/datafusion/src/datasource/file_format/mod.rs b/datafusion/src/datasource/file_format/mod.rs index b315c99b7d14..16d73e63c6a1 100644 --- a/datafusion/src/datasource/file_format/mod.rs +++ b/datafusion/src/datasource/file_format/mod.rs @@ -22,19 +22,20 @@ pub mod csv; pub mod json; pub mod parquet; -use std::pin::Pin; +use std::any::Any; +use std::fmt; use std::sync::Arc; use crate::arrow::datatypes::SchemaRef; use crate::datasource::{create_max_min_accs, get_col_stats}; use crate::error::Result; use crate::logical_plan::Expr; -use crate::physical_plan::{Accumulator, ExecutionPlan, Statistics}; +use crate::physical_plan::{ExecutionPlan, Statistics}; use async_trait::async_trait; -use futures::{Stream, StreamExt}; -use super::object_store::{FileMeta, ObjectReader, ObjectReaderStream, ObjectStore}; +use super::object_store::{ObjectReader, ObjectReaderStream, ObjectStore}; +use super::PartitionedFile; /// The configurations to be passed when creating a physical plan for /// a given file format. @@ -61,7 +62,11 @@ pub struct PhysicalPlanConfig { /// from the `TableProvider`. This helps code re-utilization accross /// providers that support the the same file formats. #[async_trait] -pub trait FileFormat: Send + Sync { +pub trait FileFormat: Send + Sync + fmt::Debug { + /// Returns the table provider as [`Any`](std::any::Any) so that it can be + /// downcast to a specific implementation. + fn as_any(&self) -> &dyn Any; + /// 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 @@ -79,128 +84,3 @@ pub trait FileFormat: Send + Sync { conf: PhysicalPlanConfig, ) -> Result>; } - -/// Get all files as well as the summary statistic -/// if the optional `limit` is provided, includes only sufficient files -/// 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 async fn get_statistics_with_limit( - all_files: impl Stream>, - schema: SchemaRef, - limit: Option, -) -> Result<(Vec, Statistics)> { - let mut result_files = vec![]; - - let mut total_byte_size = 0; - let mut null_counts = vec![0; schema.fields().len()]; - let mut has_statistics = false; - let (mut max_values, mut min_values) = create_max_min_accs(&schema); - - let mut num_rows = 0; - let mut is_exact = true; - 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); - if let Some(vec) = &file_stats.column_statistics { - has_statistics = true; - for (i, cs) in vec.iter().enumerate() { - null_counts[i] += cs.null_count.unwrap_or(0); - - if let Some(max_value) = &mut max_values[i] { - if let Some(file_max) = cs.max_value.clone() { - match max_value.update(&[file_max]) { - Ok(_) => {} - Err(_) => { - max_values[i] = None; - } - } - } - } - - if let Some(min_value) = &mut min_values[i] { - if let Some(file_min) = cs.min_value.clone() { - match min_value.update(&[file_min]) { - Ok(_) => {} - Err(_) => { - min_values[i] = None; - } - } - } - } - } - } - if num_rows > limit.unwrap_or(usize::MAX) { - break; - } - } - // 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; - } - - let column_stats = if has_statistics { - Some(get_col_stats( - &*schema, - null_counts, - &mut max_values, - &mut min_values, - )) - } else { - None - }; - - let statistics = Statistics { - num_rows: Some(num_rows as usize), - total_byte_size: Some(total_byte_size as usize), - column_statistics: column_stats, - is_exact, - }; - - Ok((result_files, statistics)) -} - -#[derive(Debug, Clone)] -/// A single file that should be read, along with its schema, statistics -/// and partition column values that need to be appended to each row. -/// 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_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 -} - -/// 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.file_meta) - } -} - -#[derive(Debug, Clone)] -/// A collection of files that should be read in a single task -/// TODO move back to crate::datasource::mod.rs once legacy cleaned up -pub struct FilePartition { - /// The index of the partition among all partitions - pub index: usize, - /// The contained files of the partition - pub files: Vec, -} - -impl std::fmt::Display for FilePartition { - fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { - let files: Vec = self.files.iter().map(|f| f.to_string()).collect(); - write!(f, "{}", files.join(", ")) - } -} diff --git a/datafusion/src/datasource/file_format/parquet.rs b/datafusion/src/datasource/file_format/parquet.rs index 10fe0d79ebeb..cd617ebc4c4d 100644 --- a/datafusion/src/datasource/file_format/parquet.rs +++ b/datafusion/src/datasource/file_format/parquet.rs @@ -17,6 +17,7 @@ //! Parquet format abstractions +use std::any::Any; use std::io::Read; use std::sync::Arc; @@ -47,7 +48,11 @@ use crate::physical_plan::ExecutionPlan; use crate::physical_plan::{Accumulator, Statistics}; use crate::scalar::ScalarValue; +/// The default file exetension of parquet files +pub const DEFAULT_PARQUET_EXTENSION: &str = ".parquet"; + /// The Apache Parquet `FileFormat` implementation +#[derive(Debug)] pub struct ParquetFormat { enable_pruning: bool, } @@ -67,10 +72,18 @@ impl ParquetFormat { self.enable_pruning = enable; self } + /// Return true if pruning is enabled + pub fn enable_pruning(&self) -> bool { + self.enable_pruning + } } #[async_trait] impl FileFormat for ParquetFormat { + fn as_any(&self) -> &dyn Any { + self + } + 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. @@ -318,11 +331,11 @@ impl ChunkReader for ChunkObjectReader { mod tests { use crate::{ datasource::{ - file_format::PartitionedFile, object_store::local::{ local_file_meta, local_object_reader, local_object_reader_stream, LocalFileSystem, }, + PartitionedFile, }, physical_plan::collect, }; diff --git a/datafusion/src/datasource/json.rs b/datafusion/src/datasource/json.rs deleted file mode 100644 index 1a6ec7af0720..000000000000 --- a/datafusion/src/datasource/json.rs +++ /dev/null @@ -1,184 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -//! Line-delimited JSON data source -//! -//! This data source allows Line-delimited JSON string or files to be used as input for queries. -//! - -use std::{ - any::Any, - io::{BufReader, Read, Seek}, - sync::{Arc, Mutex}, -}; - -use crate::{ - datasource::{Source, TableProvider}, - error::{DataFusionError, Result}, - physical_plan::{ - common, - json::{NdJsonExec, NdJsonReadOptions}, - ExecutionPlan, - }, -}; -use arrow::{datatypes::SchemaRef, json::reader::infer_json_schema_from_seekable}; -use async_trait::async_trait; - -trait SeekRead: Read + Seek {} - -impl SeekRead for T {} - -/// Represents a line-delimited JSON file with a provided schema -pub struct NdJsonFile { - source: Source>, - schema: SchemaRef, - file_extension: String, -} - -impl NdJsonFile { - /// Attempt to initialize a `NdJsonFile` from a path. The schema can be inferred automatically. - pub fn try_new(path: &str, options: NdJsonReadOptions) -> Result { - let schema = if let Some(schema) = options.schema { - schema - } else { - let filenames = common::build_file_list(path, options.file_extension)?; - if filenames.is_empty() { - return Err(DataFusionError::Plan(format!( - "No files found at {path} with file extension {file_extension}", - path = path, - file_extension = options.file_extension - ))); - } - - NdJsonExec::try_infer_schema( - filenames, - Some(options.schema_infer_max_records), - )? - .into() - }; - - Ok(Self { - source: Source::Path(path.to_string()), - schema, - file_extension: options.file_extension.to_string(), - }) - } - - /// Attempt to initialize a `NdJsonFile` from a reader impls `Seek`. The schema can be inferred automatically. - pub fn try_new_from_reader( - mut reader: R, - options: NdJsonReadOptions, - ) -> Result { - let schema = if let Some(schema) = options.schema { - schema - } else { - let mut bufr = BufReader::new(reader); - let schema = infer_json_schema_from_seekable( - &mut bufr, - Some(options.schema_infer_max_records), - )? - .into(); - reader = bufr.into_inner(); - schema - }; - Ok(Self { - source: Source::Reader(Mutex::new(Some(Box::new(reader)))), - schema, - file_extension: String::new(), - }) - } -} - -#[async_trait] -impl TableProvider for NdJsonFile { - fn as_any(&self) -> &dyn Any { - self - } - - fn schema(&self) -> SchemaRef { - self.schema.clone() - } - - async fn scan( - &self, - projection: &Option>, - batch_size: usize, - _filters: &[crate::logical_plan::Expr], - limit: Option, - ) -> Result> { - let opts = NdJsonReadOptions { - schema: Some(self.schema.clone()), - schema_infer_max_records: 0, // schema will always be provided, so it's unnecessary to infer schema - file_extension: self.file_extension.as_str(), - }; - let batch_size = limit - .map(|l| std::cmp::min(l, batch_size)) - .unwrap_or(batch_size); - - let exec = match &self.source { - Source::Reader(maybe_reader) => { - if let Some(rdr) = maybe_reader.lock().unwrap().take() { - NdJsonExec::try_new_from_reader( - rdr, - opts, - projection.clone(), - batch_size, - limit, - )? - } else { - return Err(DataFusionError::Execution( - "You can only read once if the data comes from a reader" - .to_string(), - )); - } - } - Source::Path(p) => { - NdJsonExec::try_new(p, opts, projection.clone(), batch_size, limit)? - } - }; - Ok(Arc::new(exec)) - } -} - -#[cfg(test)] -mod tests { - use super::*; - use crate::prelude::*; - const TEST_DATA_BASE: &str = "tests/jsons"; - - #[tokio::test] - async fn csv_file_from_reader() -> Result<()> { - let mut ctx = ExecutionContext::new(); - let path = format!("{}/2.json", TEST_DATA_BASE); - ctx.register_table( - "ndjson", - Arc::new(NdJsonFile::try_new(&path, Default::default())?), - )?; - let df = ctx.sql("select sum(a) from ndjson")?; - let batches = df.collect().await?; - assert_eq!( - batches[0] - .column(0) - .as_any() - .downcast_ref::() - .unwrap() - .value(0), - 100000000000011 - ); - Ok(()) - } -} diff --git a/datafusion/src/datasource/listing.rs b/datafusion/src/datasource/listing.rs index 1e048e12d98d..959418f4ea02 100644 --- a/datafusion/src/datasource/listing.rs +++ b/datafusion/src/datasource/listing.rs @@ -25,7 +25,7 @@ use async_trait::async_trait; use futures::StreamExt; use crate::{ - datasource::file_format::{self, PartitionedFile}, + datasource::PartitionedFile, error::{DataFusionError, Result}, logical_plan::Expr, physical_plan::{ExecutionPlan, Statistics}, @@ -33,9 +33,10 @@ use crate::{ use super::{ datasource::TableProviderFilterPushDown, - file_format::{FileFormat, PartitionedFileStream, PhysicalPlanConfig}, + file_format::{FileFormat, PhysicalPlanConfig}, + get_statistics_with_limit, object_store::ObjectStore, - TableProvider, + PartitionedFileStream, TableProvider, }; /// Options for creating a `ListingTable` @@ -59,10 +60,26 @@ pub struct ListingOptions { pub collect_stat: bool, /// Group files to avoid that the number of partitions /// exceeds this limit - pub max_partitions: usize, + pub target_partitions: usize, } impl ListingOptions { + /// Creates an options instance with the given format + /// Default values: + /// - no file extension filter + /// - no input partition to discover + /// - one target partition + /// - no stat collection + pub fn new(format: Arc) -> Self { + Self { + file_extension: String::new(), + format, + partitions: vec![], + collect_stat: false, + target_partitions: 1, + } + } + /// Infer the schema of the files at the given uri, including the partitioning /// columns. /// @@ -113,6 +130,19 @@ impl ListingTable { options, } } + + /// Get object store ref + pub fn object_store(&self) -> &Arc { + &self.object_store + } + /// Get path ref + pub fn path(&self) -> &str { + &self.path + } + /// Get options ref + pub fn options(&self) -> &ListingOptions { + &self.options + } } #[async_trait] @@ -200,7 +230,7 @@ impl ListingTable { }); let (files, statistics) = - file_format::get_statistics_with_limit(files, self.schema(), limit).await?; + get_statistics_with_limit(files, self.schema(), limit).await?; if files.is_empty() { return Err(DataFusionError::Plan(format!( @@ -209,7 +239,10 @@ impl ListingTable { ))); } - Ok((split_files(files, self.options.max_partitions), statistics)) + Ok(( + split_files(files, self.options.target_partitions), + statistics, + )) } } @@ -344,7 +377,7 @@ mod tests { file_extension: "parquet".to_owned(), format: Arc::new(ParquetFormat::default()), partitions: vec![], - max_partitions: 2, + target_partitions: 2, collect_stat: true, }; // here we resolve the schema locally @@ -359,7 +392,7 @@ mod tests { async fn assert_partitioning( files_in_folder: usize, - max_partitions: usize, + target_partitions: usize, output_partitioning: usize, ) -> Result<()> { let mock_store: Arc = @@ -371,7 +404,7 @@ mod tests { file_extension: "".to_owned(), format: Arc::new(format), partitions: vec![], - max_partitions, + target_partitions, collect_stat: true, }; diff --git a/datafusion/src/datasource/mod.rs b/datafusion/src/datasource/mod.rs index 299991a5d64f..31fde168b046 100644 --- a/datafusion/src/datasource/mod.rs +++ b/datafusion/src/datasource/mod.rs @@ -17,177 +17,36 @@ //! DataFusion data sources -pub mod avro; -pub mod csv; pub mod datasource; pub mod empty; pub mod file_format; -pub mod json; pub mod listing; pub mod memory; pub mod object_store; -pub mod parquet; -pub use self::csv::{CsvFile, CsvReadOptions}; +use futures::Stream; + pub use self::datasource::{TableProvider, TableType}; pub use self::memory::MemTable; +use self::object_store::{FileMeta, SizedFile}; use crate::arrow::datatypes::{Schema, SchemaRef}; -use crate::error::{DataFusionError, Result}; -use crate::physical_plan::common::build_file_list; +use crate::error::Result; use crate::physical_plan::expressions::{MaxAccumulator, MinAccumulator}; use crate::physical_plan::{Accumulator, ColumnStatistics, Statistics}; -use std::sync::Arc; - -/// Source for table input data -pub(crate) enum Source> { - /// Path to a single file or a directory containing one of more files - Path(String), - - /// Read data from a reader - Reader(std::sync::Mutex>), -} - -#[derive(Debug, Clone)] -/// A single file that should be read, along with its schema, statistics -/// and partition column values that need to be appended to each row. -pub struct PartitionedFile { - /// Path for the file (e.g. URL, filesystem path, etc) - pub path: String, - /// Statistics of the file - pub statistics: Statistics, - // 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 -} - -impl From for PartitionedFile { - fn from(path: String) -> Self { - Self { - path, - statistics: Default::default(), - } - } -} - -impl std::fmt::Display for PartitionedFile { - fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { - write!(f, "{}", self.path) - } -} - -#[derive(Debug, Clone)] -/// A collection of files that should be read in a single task -pub struct FilePartition { - /// The index of the partition among all partitions - pub index: usize, - /// The contained files of the partition - pub files: Vec, -} - -impl std::fmt::Display for FilePartition { - fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { - let files: Vec = self.files.iter().map(|f| f.to_string()).collect(); - write!(f, "{}", files.join(", ")) - } -} - -#[derive(Debug, Clone)] -/// All source files with same schema exists in a path -pub struct TableDescriptor { - /// root path of the table - pub path: String, - /// All source files in the path - pub partition_files: Vec, - /// The schema of the files - pub schema: SchemaRef, -} - -/// Returned partitioned file with its schema -pub struct FileAndSchema { - file: PartitionedFile, - schema: Schema, -} - -/// Builder for ['TableDescriptor'] inside given path -pub trait TableDescriptorBuilder { - /// Construct a ['TableDescriptor'] from the provided path - fn build_table_desc( - path: &str, - ext: &str, - provided_schema: Option, - collect_statistics: bool, - ) -> Result { - let filenames = build_file_list(path, ext)?; - if filenames.is_empty() { - return Err(DataFusionError::Plan(format!( - "No file (with .{} extension) found at path {}", - ext, path - ))); - } - - // build a list of partitions with statistics and gather all unique schemas - // used in this data set - let mut schemas: Vec = vec![]; - let mut contains_file = false; - - let partitioned_files = filenames - .iter() - .map(|file_path| { - contains_file = true; - let result = if collect_statistics { - let FileAndSchema {file, schema} = Self::file_meta(file_path)?; - if schemas.is_empty() { - schemas.push(schema); - } else if schema.fields() != schemas[0].fields() { - // 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 - return Err(DataFusionError::Plan(format!( - "The file {} have different schema from the first file and DataFusion does \ - not yet support schema merging", - file_path - ))); - } - file - } else { - PartitionedFile { - path: file_path.to_owned(), - statistics: Statistics::default(), - } - }; - - Ok(result) - }).collect::>>(); - - if !contains_file { - return Err(DataFusionError::Plan(format!( - "No file (with .{} extension) found at path {}", - ext, path - ))); - } - - let result_schema = provided_schema.unwrap_or_else(|| schemas.pop().unwrap()); - - Ok(TableDescriptor { - path: path.to_string(), - partition_files: partitioned_files?, - schema: Arc::new(result_schema), - }) - } - - /// Get all metadata for a source file, including schema, statistics, partitions, etc. - fn file_meta(path: &str) -> Result; -} +use futures::StreamExt; +use std::pin::Pin; /// Get all files as well as the summary statistic /// if the optional `limit` is provided, includes only sufficient files /// needed to read up to `limit` number of rows -pub fn get_statistics_with_limit( - table_desc: &TableDescriptor, +/// 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 async fn get_statistics_with_limit( + all_files: impl Stream>, + schema: SchemaRef, limit: Option, -) -> (Vec, Statistics) { - let mut all_files = table_desc.partition_files.clone(); - let schema = table_desc.schema.clone(); +) -> Result<(Vec, Statistics)> { + let mut result_files = vec![]; let mut total_byte_size = 0; let mut null_counts = vec![0; schema.fields().len()]; @@ -195,11 +54,12 @@ 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 in &all_files { - num_files += 1; - let file_stats = &file.statistics; + // fusing the stream allows us to call next safely even once it is finished + let mut all_files = Box::pin(all_files.fuse()); + 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); @@ -235,9 +95,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 { @@ -257,7 +119,59 @@ pub fn get_statistics_with_limit( column_statistics: column_stats, is_exact, }; - (all_files, statistics) + + Ok((result_files, statistics)) +} + +#[derive(Debug, Clone)] +/// A single file that should be read, along with its schema, statistics +/// and partition column values that need to be appended to each row. +/// 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_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 +} + +impl PartitionedFile { + /// Create a simple file without metadata or partition + pub fn new(path: String, size: u64) -> Self { + Self { + file_meta: FileMeta { + sized_file: SizedFile { path, size }, + last_modified: None, + }, + } + } +} + +/// 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.file_meta) + } +} + +#[derive(Debug, Clone)] +/// A collection of files that should be read in a single task +/// TODO move back to crate::datasource::mod.rs once legacy cleaned up +pub struct FilePartition { + /// The index of the partition among all partitions + pub index: usize, + /// The contained files of the partition + pub files: Vec, +} + +impl std::fmt::Display for FilePartition { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + let files: Vec = self.files.iter().map(|f| f.to_string()).collect(); + write!(f, "{}", files.join(", ")) + } } fn create_max_min_accs( diff --git a/datafusion/src/datasource/parquet.rs b/datafusion/src/datasource/parquet.rs deleted file mode 100644 index d044ed94d59d..000000000000 --- a/datafusion/src/datasource/parquet.rs +++ /dev/null @@ -1,677 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -//! Parquet data source - -use std::any::Any; -use std::fs::File; -use std::sync::Arc; - -use async_trait::async_trait; -use parquet::arrow::ArrowReader; -use parquet::arrow::ParquetFileArrowReader; -use parquet::file::serialized_reader::SerializedFileReader; -use parquet::file::statistics::Statistics as ParquetStatistics; - -use super::datasource::TableProviderFilterPushDown; -use crate::arrow::datatypes::{DataType, Field, Schema, SchemaRef}; -use crate::datasource::{ - create_max_min_accs, get_col_stats, FileAndSchema, PartitionedFile, TableDescriptor, - TableDescriptorBuilder, TableProvider, -}; -use crate::error::Result; -use crate::logical_plan::{combine_filters, Expr}; -use crate::physical_plan::expressions::{MaxAccumulator, MinAccumulator}; -use crate::physical_plan::parquet::ParquetExec; -use crate::physical_plan::{Accumulator, ExecutionPlan, Statistics}; -use crate::scalar::ScalarValue; - -/// Table-based representation of a `ParquetFile`. -pub struct ParquetTable { - /// Descriptor of the table, including schema, files, etc. - pub desc: Arc, - target_partitions: usize, - enable_pruning: bool, -} - -impl ParquetTable { - /// Attempt to initialize a new `ParquetTable` from a file path. - pub fn try_new(path: impl Into, target_partitions: usize) -> Result { - let path = path.into(); - let table_desc = ParquetTableDescriptor::new(path.as_str()); - Ok(Self { - desc: Arc::new(table_desc?), - target_partitions, - enable_pruning: true, - }) - } - - /// Attempt to initialize a new `ParquetTable` from a file path and known schema. - /// If collect_statistics is `false`, doesn't read files until necessary by scan - pub fn try_new_with_schema( - path: impl Into, - schema: Schema, - target_partitions: usize, - collect_statistics: bool, - ) -> Result { - let path = path.into(); - let table_desc = ParquetTableDescriptor::new_with_schema( - path.as_str(), - Some(schema), - collect_statistics, - ); - Ok(Self { - desc: Arc::new(table_desc?), - target_partitions, - enable_pruning: true, - }) - } - - /// Attempt to initialize a new `ParquetTable` from a table descriptor. - pub fn try_new_with_desc( - desc: Arc, - target_partitions: usize, - enable_pruning: bool, - ) -> Result { - Ok(Self { - desc, - target_partitions, - enable_pruning, - }) - } - - /// Get the path for the Parquet file(s) represented by this ParquetTable instance - pub fn path(&self) -> &str { - &self.desc.descriptor.path - } - - /// Get parquet pruning option - pub fn get_enable_pruning(&self) -> bool { - self.enable_pruning - } - - /// Set parquet pruning option - pub fn with_enable_pruning(mut self, enable_pruning: bool) -> Self { - self.enable_pruning = enable_pruning; - self - } - - /// Get Target partitions - pub fn get_target_partitions(&self) -> usize { - self.target_partitions - } -} - -#[async_trait] -impl TableProvider for ParquetTable { - fn as_any(&self) -> &dyn Any { - self - } - - /// Get the schema for this parquet file. - fn schema(&self) -> SchemaRef { - self.desc.schema() - } - - fn supports_filter_pushdown( - &self, - _filter: &Expr, - ) -> Result { - Ok(TableProviderFilterPushDown::Inexact) - } - - /// Scan the file(s), using the provided projection, and return one BatchIterator per - /// partition. - async fn scan( - &self, - projection: &Option>, - batch_size: usize, - filters: &[Expr], - limit: Option, - ) -> 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) - } else { - None - }; - Ok(Arc::new(ParquetExec::try_new( - self.desc.clone(), - projection.clone(), - predicate, - limit - .map(|l| std::cmp::min(l, batch_size)) - .unwrap_or(batch_size), - self.target_partitions, - limit, - )?)) - } -} - -#[derive(Debug, Clone)] -/// Descriptor for a parquet root path -pub struct ParquetTableDescriptor { - /// metadata for files inside the root path - pub descriptor: TableDescriptor, -} - -impl ParquetTableDescriptor { - /// Construct a new parquet descriptor for a root path - pub fn new(root_path: &str) -> Result { - let table_desc = Self::build_table_desc(root_path, "parquet", None, true); - Ok(Self { - descriptor: table_desc?, - }) - } - - /// Construct a new parquet descriptor for a root path with known schema - pub fn new_with_schema( - root_path: &str, - schema: Option, - collect_statistics: bool, - ) -> Result { - let table_desc = - Self::build_table_desc(root_path, "parquet", schema, collect_statistics); - Ok(Self { - descriptor: table_desc?, - }) - } - - /// Get file schema for all parquet files - pub fn schema(&self) -> SchemaRef { - self.descriptor.schema.clone() - } - - fn summarize_min_max( - max_values: &mut Vec>, - min_values: &mut Vec>, - fields: &[Field], - i: usize, - stat: &ParquetStatistics, - ) { - match stat { - ParquetStatistics::Boolean(s) => { - if let DataType::Boolean = fields[i].data_type() { - if s.has_min_max_set() { - if let Some(max_value) = &mut max_values[i] { - match max_value - .update(&[ScalarValue::Boolean(Some(*s.max()))]) - { - Ok(_) => {} - Err(_) => { - max_values[i] = None; - } - } - } - if let Some(min_value) = &mut min_values[i] { - match min_value - .update(&[ScalarValue::Boolean(Some(*s.min()))]) - { - Ok(_) => {} - Err(_) => { - min_values[i] = None; - } - } - } - } - } - } - ParquetStatistics::Int32(s) => { - if let DataType::Int32 = fields[i].data_type() { - if s.has_min_max_set() { - if let Some(max_value) = &mut max_values[i] { - match max_value.update(&[ScalarValue::Int32(Some(*s.max()))]) - { - Ok(_) => {} - Err(_) => { - max_values[i] = None; - } - } - } - if let Some(min_value) = &mut min_values[i] { - match min_value.update(&[ScalarValue::Int32(Some(*s.min()))]) - { - Ok(_) => {} - Err(_) => { - min_values[i] = None; - } - } - } - } - } - } - ParquetStatistics::Int64(s) => { - if let DataType::Int64 = fields[i].data_type() { - if s.has_min_max_set() { - if let Some(max_value) = &mut max_values[i] { - match max_value.update(&[ScalarValue::Int64(Some(*s.max()))]) - { - Ok(_) => {} - Err(_) => { - max_values[i] = None; - } - } - } - if let Some(min_value) = &mut min_values[i] { - match min_value.update(&[ScalarValue::Int64(Some(*s.min()))]) - { - Ok(_) => {} - Err(_) => { - min_values[i] = None; - } - } - } - } - } - } - ParquetStatistics::Float(s) => { - if let DataType::Float32 = fields[i].data_type() { - if s.has_min_max_set() { - if let Some(max_value) = &mut max_values[i] { - match max_value - .update(&[ScalarValue::Float32(Some(*s.max()))]) - { - Ok(_) => {} - Err(_) => { - max_values[i] = None; - } - } - } - if let Some(min_value) = &mut min_values[i] { - match min_value - .update(&[ScalarValue::Float32(Some(*s.min()))]) - { - Ok(_) => {} - Err(_) => { - min_values[i] = None; - } - } - } - } - } - } - ParquetStatistics::Double(s) => { - if let DataType::Float64 = fields[i].data_type() { - if s.has_min_max_set() { - if let Some(max_value) = &mut max_values[i] { - match max_value - .update(&[ScalarValue::Float64(Some(*s.max()))]) - { - Ok(_) => {} - Err(_) => { - max_values[i] = None; - } - } - } - if let Some(min_value) = &mut min_values[i] { - match min_value - .update(&[ScalarValue::Float64(Some(*s.min()))]) - { - Ok(_) => {} - Err(_) => { - min_values[i] = None; - } - } - } - } - } - } - _ => {} - } - } -} - -impl TableDescriptorBuilder for ParquetTableDescriptor { - fn file_meta(path: &str) -> Result { - let file = File::open(path)?; - let file_reader = Arc::new(SerializedFileReader::new(file)?); - let mut arrow_reader = ParquetFileArrowReader::new(file_reader); - let path = path.to_string(); - let schema = arrow_reader.get_schema()?; - let num_fields = schema.fields().len(); - let fields = schema.fields().to_vec(); - let meta_data = arrow_reader.get_metadata(); - - let mut num_rows = 0; - let mut total_byte_size = 0; - let mut null_counts = vec![0; num_fields]; - let mut has_statistics = false; - - let (mut max_values, mut min_values) = create_max_min_accs(&schema); - - for row_group_meta in meta_data.row_groups() { - num_rows += row_group_meta.num_rows(); - total_byte_size += row_group_meta.total_byte_size(); - - let columns_null_counts = row_group_meta - .columns() - .iter() - .flat_map(|c| c.statistics().map(|stats| stats.null_count())); - - for (i, cnt) in columns_null_counts.enumerate() { - null_counts[i] += cnt as usize - } - - for (i, column) in row_group_meta.columns().iter().enumerate() { - if let Some(stat) = column.statistics() { - has_statistics = true; - ParquetTableDescriptor::summarize_min_max( - &mut max_values, - &mut min_values, - &fields, - i, - stat, - ) - } - } - } - - let column_stats = if has_statistics { - Some(get_col_stats( - &schema, - null_counts, - &mut max_values, - &mut min_values, - )) - } else { - None - }; - - let statistics = Statistics { - num_rows: Some(num_rows as usize), - total_byte_size: Some(total_byte_size as usize), - column_statistics: column_stats, - is_exact: true, - }; - - Ok(FileAndSchema { - file: PartitionedFile { path, statistics }, - schema, - }) - } -} - -#[cfg(test)] -mod tests { - use super::*; - use arrow::array::{ - BinaryArray, BooleanArray, Float32Array, Float64Array, Int32Array, - TimestampNanosecondArray, - }; - use arrow::record_batch::RecordBatch; - use futures::StreamExt; - - #[tokio::test] - async fn read_small_batches() -> Result<()> { - let table = load_table("alltypes_plain.parquet")?; - let projection = None; - let exec = table.scan(&projection, 2, &[], None).await?; - let stream = exec.execute(0).await?; - - let _ = stream - .map(|batch| { - let batch = batch.unwrap(); - assert_eq!(11, batch.num_columns()); - assert_eq!(2, batch.num_rows()); - }) - .fold(0, |acc, _| async move { acc + 1i32 }) - .await; - - // test metadata - assert_eq!(exec.statistics().num_rows, Some(8)); - assert_eq!(exec.statistics().total_byte_size, Some(671)); - - Ok(()) - } - - #[tokio::test] - async fn read_alltypes_plain_parquet() -> Result<()> { - let table = load_table("alltypes_plain.parquet")?; - - let x: Vec = table - .schema() - .fields() - .iter() - .map(|f| format!("{}: {:?}", f.name(), f.data_type())) - .collect(); - let y = x.join("\n"); - assert_eq!( - "id: Int32\n\ - bool_col: Boolean\n\ - tinyint_col: Int32\n\ - smallint_col: Int32\n\ - int_col: Int32\n\ - bigint_col: Int64\n\ - float_col: Float32\n\ - double_col: Float64\n\ - date_string_col: Binary\n\ - string_col: Binary\n\ - timestamp_col: Timestamp(Nanosecond, None)", - y - ); - - let projection = None; - let batch = get_first_batch(table, &projection).await?; - - assert_eq!(11, batch.num_columns()); - assert_eq!(8, batch.num_rows()); - - Ok(()) - } - - #[tokio::test] - async fn read_bool_alltypes_plain_parquet() -> Result<()> { - let table = load_table("alltypes_plain.parquet")?; - let projection = Some(vec![1]); - let batch = get_first_batch(table, &projection).await?; - - assert_eq!(1, batch.num_columns()); - assert_eq!(8, batch.num_rows()); - - let array = batch - .column(0) - .as_any() - .downcast_ref::() - .unwrap(); - let mut values: Vec = vec![]; - for i in 0..batch.num_rows() { - values.push(array.value(i)); - } - - assert_eq!( - "[true, false, true, false, true, false, true, false]", - format!("{:?}", values) - ); - - Ok(()) - } - - #[tokio::test] - async fn read_i32_alltypes_plain_parquet() -> Result<()> { - let table = load_table("alltypes_plain.parquet")?; - let projection = Some(vec![0]); - let batch = get_first_batch(table, &projection).await?; - - assert_eq!(1, batch.num_columns()); - assert_eq!(8, batch.num_rows()); - - let array = batch - .column(0) - .as_any() - .downcast_ref::() - .unwrap(); - let mut values: Vec = vec![]; - for i in 0..batch.num_rows() { - values.push(array.value(i)); - } - - assert_eq!("[4, 5, 6, 7, 2, 3, 0, 1]", format!("{:?}", values)); - - Ok(()) - } - - #[tokio::test] - async fn read_i96_alltypes_plain_parquet() -> Result<()> { - let table = load_table("alltypes_plain.parquet")?; - let projection = Some(vec![10]); - let batch = get_first_batch(table, &projection).await?; - - assert_eq!(1, batch.num_columns()); - assert_eq!(8, batch.num_rows()); - - let array = batch - .column(0) - .as_any() - .downcast_ref::() - .unwrap(); - let mut values: Vec = vec![]; - for i in 0..batch.num_rows() { - values.push(array.value(i)); - } - - assert_eq!("[1235865600000000000, 1235865660000000000, 1238544000000000000, 1238544060000000000, 1233446400000000000, 1233446460000000000, 1230768000000000000, 1230768060000000000]", format!("{:?}", values)); - - Ok(()) - } - - #[tokio::test] - async fn read_f32_alltypes_plain_parquet() -> Result<()> { - let table = load_table("alltypes_plain.parquet")?; - let projection = Some(vec![6]); - let batch = get_first_batch(table, &projection).await?; - - assert_eq!(1, batch.num_columns()); - assert_eq!(8, batch.num_rows()); - - let array = batch - .column(0) - .as_any() - .downcast_ref::() - .unwrap(); - let mut values: Vec = vec![]; - for i in 0..batch.num_rows() { - values.push(array.value(i)); - } - - assert_eq!( - "[0.0, 1.1, 0.0, 1.1, 0.0, 1.1, 0.0, 1.1]", - format!("{:?}", values) - ); - - Ok(()) - } - - #[tokio::test] - async fn read_f64_alltypes_plain_parquet() -> Result<()> { - let table = load_table("alltypes_plain.parquet")?; - let projection = Some(vec![7]); - let batch = get_first_batch(table, &projection).await?; - - assert_eq!(1, batch.num_columns()); - assert_eq!(8, batch.num_rows()); - - let array = batch - .column(0) - .as_any() - .downcast_ref::() - .unwrap(); - let mut values: Vec = vec![]; - for i in 0..batch.num_rows() { - values.push(array.value(i)); - } - - assert_eq!( - "[0.0, 10.1, 0.0, 10.1, 0.0, 10.1, 0.0, 10.1]", - format!("{:?}", values) - ); - - Ok(()) - } - - #[tokio::test] - async fn read_binary_alltypes_plain_parquet() -> Result<()> { - let table = load_table("alltypes_plain.parquet")?; - let projection = Some(vec![9]); - let batch = get_first_batch(table, &projection).await?; - - assert_eq!(1, batch.num_columns()); - assert_eq!(8, batch.num_rows()); - - let array = batch - .column(0) - .as_any() - .downcast_ref::() - .unwrap(); - let mut values: Vec<&str> = vec![]; - for i in 0..batch.num_rows() { - values.push(std::str::from_utf8(array.value(i)).unwrap()); - } - - assert_eq!( - "[\"0\", \"1\", \"0\", \"1\", \"0\", \"1\", \"0\", \"1\"]", - format!("{:?}", values) - ); - - Ok(()) - } - - fn load_table(name: &str) -> Result> { - let testdata = crate::test_util::parquet_test_data(); - let filename = format!("{}/{}", testdata, name); - let table = ParquetTable::try_new(&filename, 2)?; - Ok(Arc::new(table)) - } - - async fn get_first_batch( - table: Arc, - projection: &Option>, - ) -> Result { - let exec = table.scan(projection, 1024, &[], None).await?; - let mut it = exec.execute(0).await?; - it.next() - .await - .expect("should have received at least one batch") - .map_err(|e| e.into()) - } - - #[test] - fn combine_zero_filters() { - let result = combine_filters(&[]); - assert_eq!(result, None); - } - - #[test] - fn combine_one_filter() { - use crate::logical_plan::{binary_expr, col, lit, Operator}; - let filter = binary_expr(col("c1"), Operator::Lt, lit(1)); - let result = combine_filters(&[filter.clone()]); - assert_eq!(result, Some(filter)); - } - - #[test] - fn combine_multiple_filters() { - use crate::logical_plan::{and, binary_expr, col, lit, Operator}; - let filter1 = binary_expr(col("c1"), Operator::Lt, lit(1)); - let filter2 = binary_expr(col("c2"), Operator::Lt, lit(2)); - let filter3 = binary_expr(col("c3"), Operator::Lt, lit(3)); - let result = - combine_filters(&[filter1.clone(), filter2.clone(), filter3.clone()]); - assert_eq!(result, Some(and(and(filter1, filter2), filter3))); - } -} diff --git a/datafusion/src/execution/context.rs b/datafusion/src/execution/context.rs index 7272e57c5693..c8331da6f3b7 100644 --- a/datafusion/src/execution/context.rs +++ b/datafusion/src/execution/context.rs @@ -21,6 +21,13 @@ use crate::{ catalog::{CatalogList, MemoryCatalogList}, information_schema::CatalogWithInformationSchema, }, + datasource::file_format::{ + avro::AvroFormat, + csv::CsvFormat, + parquet::{ParquetFormat, DEFAULT_PARQUET_EXTENSION}, + FileFormat, + }, + datasource::listing::{ListingOptions, ListingTable}, logical_plan::{PlanType, ToStringifiedPlan}, optimizer::eliminate_limit::EliminateLimit, physical_optimizer::{ @@ -41,16 +48,14 @@ use std::{ use futures::{StreamExt, TryStreamExt}; use tokio::task::{self, JoinHandle}; -use arrow::csv; +use arrow::{csv, datatypes::SchemaRef}; use crate::catalog::{ catalog::{CatalogProvider, MemoryCatalogProvider}, schema::{MemorySchemaProvider, SchemaProvider}, ResolvedTableReference, TableReference, }; -use crate::datasource::csv::CsvFile; use crate::datasource::object_store::{ObjectStore, ObjectStoreRegistry}; -use crate::datasource::parquet::ParquetTable; use crate::datasource::TableProvider; use crate::error::{DataFusionError, Result}; use crate::execution::dataframe_impl::DataFrameImpl; @@ -68,9 +73,6 @@ use crate::physical_optimizer::coalesce_batches::CoalesceBatches; use crate::physical_optimizer::merge_exec::AddCoalescePartitionsExec; use crate::physical_optimizer::repartition::Repartition; -use crate::datasource::avro::AvroFile; -use crate::physical_plan::avro::AvroReadOptions; -use crate::physical_plan::csv::CsvReadOptions; use crate::physical_plan::planner::DefaultPhysicalPlanner; use crate::physical_plan::udf::ScalarUDF; use crate::physical_plan::ExecutionPlan; @@ -86,6 +88,8 @@ use chrono::{DateTime, Utc}; use parquet::arrow::ArrowWriter; use parquet::file::properties::WriterProperties; +use super::options::{AvroReadOptions, CsvReadOptions}; + /// ExecutionContext is the main interface for executing queries with DataFusion. The context /// provides the following functionality: /// @@ -100,9 +104,10 @@ use parquet::file::properties::WriterProperties; /// ``` /// use datafusion::prelude::*; /// # use datafusion::error::Result; -/// # fn main() -> Result<()> { +/// # #[tokio::main] +/// # async fn main() -> Result<()> { /// let mut ctx = ExecutionContext::new(); -/// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new())?; +/// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new()).await?; /// let df = df.filter(col("a").lt_eq(col("b")))? /// .aggregate(vec![col("a")], vec![min(col("b"))])? /// .limit(100)?; @@ -117,10 +122,11 @@ use parquet::file::properties::WriterProperties; /// use datafusion::prelude::*; /// /// # use datafusion::error::Result; -/// # fn main() -> Result<()> { +/// # #[tokio::main] +/// # async fn main() -> Result<()> { /// let mut ctx = ExecutionContext::new(); -/// ctx.register_csv("example", "tests/example.csv", CsvReadOptions::new())?; -/// let results = ctx.sql("SELECT a, MIN(b) FROM example GROUP BY a LIMIT 100")?; +/// ctx.register_csv("example", "tests/example.csv", CsvReadOptions::new()).await?; +/// let results = ctx.sql("SELECT a, MIN(b) FROM example GROUP BY a LIMIT 100").await?; /// # Ok(()) /// # } /// ``` @@ -175,7 +181,9 @@ impl ExecutionContext { } /// Creates a dataframe that will execute a SQL query. - pub fn sql(&mut self, sql: &str) -> Result> { + /// + /// async because CreateExternalTable queries might need schema inference + pub async fn sql(&mut self, sql: &str) -> Result> { let plan = self.create_logical_plan(sql)?; match plan { LogicalPlan::CreateExternalTable { @@ -184,32 +192,49 @@ impl ExecutionContext { ref location, ref file_type, ref has_header, - } => match file_type { - FileType::CSV => { - let mut options = CsvReadOptions::new().has_header(*has_header); - let tmp_schema = schema.as_ref().to_owned().into(); - if !schema.fields().is_empty() { - options = options.schema(&tmp_schema); + } => { + let file_format = match file_type { + FileType::CSV => { + Ok(Arc::new(CsvFormat::default().with_has_header(*has_header)) + as Arc) } - self.register_csv(name, location, options)?; - let plan = LogicalPlanBuilder::empty(false).build()?; - Ok(Arc::new(DataFrameImpl::new(self.state.clone(), &plan))) - } - FileType::Parquet => { - self.register_parquet(name, location)?; - let plan = LogicalPlanBuilder::empty(false).build()?; - Ok(Arc::new(DataFrameImpl::new(self.state.clone(), &plan))) - } - FileType::Avro => { - self.register_avro(name, location, AvroReadOptions::default())?; - let plan = LogicalPlanBuilder::empty(false).build()?; - Ok(Arc::new(DataFrameImpl::new(self.state.clone(), &plan))) - } - _ => Err(DataFusionError::NotImplemented(format!( - "Unsupported file type {:?}.", - file_type - ))), - }, + FileType::Parquet => { + Ok(Arc::new(ParquetFormat::default()) as Arc) + } + FileType::Avro => { + Ok(Arc::new(AvroFormat::default()) as Arc) + } + _ => Err(DataFusionError::NotImplemented(format!( + "Unsupported file type {:?}.", + file_type + ))), + }?; + + let options = ListingOptions { + format: file_format, + collect_stat: false, + file_extension: String::new(), + target_partitions: self + .state + .lock() + .unwrap() + .config + .target_partitions, + partitions: vec![], + }; + + // TODO make schema in CreateExternalTable optional instead of empty + let provided_schema = if schema.fields().is_empty() { + None + } else { + Some(Arc::new(schema.as_ref().to_owned().into())) + }; + + self.register_listing_table(name, location, options, provided_schema) + .await?; + let plan = LogicalPlanBuilder::empty(false).build()?; + Ok(Arc::new(DataFrameImpl::new(self.state.clone(), &plan))) + } plan => Ok(Arc::new(DataFrameImpl::new( self.state.clone(), @@ -281,42 +306,67 @@ impl ExecutionContext { /// Creates a DataFrame for reading an Avro data source. - pub fn read_avro( + pub async fn read_avro( &mut self, - filename: impl Into, - options: AvroReadOptions, + uri: impl Into, + options: AvroReadOptions<'_>, ) -> Result> { + let uri: String = uri.into(); + let (object_store, path) = self.object_store(&uri)?; Ok(Arc::new(DataFrameImpl::new( self.state.clone(), - &LogicalPlanBuilder::scan_avro(filename, options, None)?.build()?, + &LogicalPlanBuilder::scan_avro( + object_store, + path, + options, + None, + self.state.lock().unwrap().config.target_partitions, + ) + .await? + .build()?, ))) } /// Creates a DataFrame for reading a CSV data source. - pub fn read_csv( + pub async fn read_csv( &mut self, - filename: impl Into, - options: CsvReadOptions, + uri: impl Into, + options: CsvReadOptions<'_>, ) -> Result> { + let uri: String = uri.into(); + let (object_store, path) = self.object_store(&uri)?; Ok(Arc::new(DataFrameImpl::new( self.state.clone(), - &LogicalPlanBuilder::scan_csv(filename, options, None)?.build()?, + &LogicalPlanBuilder::scan_csv( + object_store, + path, + options, + None, + self.state.lock().unwrap().config.target_partitions, + ) + .await? + .build()?, ))) } /// Creates a DataFrame for reading a Parquet data source. - pub fn read_parquet( + pub async fn read_parquet( &mut self, - filename: impl Into, + uri: impl Into, ) -> Result> { + let uri: String = uri.into(); + let (object_store, path) = self.object_store(&uri)?; + let logical_plan = LogicalPlanBuilder::scan_parquet( + object_store, + path, + None, + self.state.lock().unwrap().config.target_partitions, + ) + .await? + .build()?; Ok(Arc::new(DataFrameImpl::new( self.state.clone(), - &LogicalPlanBuilder::scan_parquet( - filename, - None, - self.state.lock().unwrap().config.target_partitions, - )? - .build()?, + &logical_plan, ))) } @@ -331,39 +381,88 @@ impl ExecutionContext { ))) } + /// Registers a table that uses the listing feature of the object store to + /// find the files to be processed + /// This is async because it might need to resolve the schema. + pub async fn register_listing_table<'a>( + &'a mut self, + name: &'a str, + uri: &'a str, + options: ListingOptions, + provided_schema: Option, + ) -> Result<()> { + let (object_store, path) = self.object_store(uri)?; + let resolved_schema = match provided_schema { + None => { + options + .infer_schema(Arc::clone(&object_store), path) + .await? + } + Some(s) => s, + }; + let table = + ListingTable::new(object_store, path.to_owned(), resolved_schema, options); + self.register_table(name, Arc::new(table))?; + Ok(()) + } + /// Registers a CSV data source so that it can be referenced from SQL statements /// executed against this context. - pub fn register_csv( + pub async fn register_csv( &mut self, name: &str, - filename: &str, - options: CsvReadOptions, + uri: &str, + options: CsvReadOptions<'_>, ) -> Result<()> { - self.register_table(name, Arc::new(CsvFile::try_new(filename, options)?))?; + let listing_options = options + .to_listing_options(self.state.lock().unwrap().config.target_partitions); + + self.register_listing_table( + name, + uri, + listing_options, + options.schema.map(|s| Arc::new(s.to_owned())), + ) + .await?; + Ok(()) } /// Registers a Parquet data source so that it can be referenced from SQL statements /// executed against this context. - pub fn register_parquet(&mut self, name: &str, filename: &str) -> Result<()> { - let table = { + pub async fn register_parquet(&mut self, name: &str, uri: &str) -> Result<()> { + let (target_partitions, enable_pruning) = { let m = self.state.lock().unwrap(); - ParquetTable::try_new(filename, m.config.target_partitions)? - .with_enable_pruning(m.config.parquet_pruning) + (m.config.target_partitions, m.config.parquet_pruning) }; - self.register_table(name, Arc::new(table))?; + let file_format = ParquetFormat::default().with_enable_pruning(enable_pruning); + + let listing_options = ListingOptions { + format: Arc::new(file_format), + collect_stat: true, + file_extension: DEFAULT_PARQUET_EXTENSION.to_owned(), + target_partitions, + partitions: vec![], + }; + + self.register_listing_table(name, uri, listing_options, None) + .await?; Ok(()) } /// Registers an Avro data source so that it can be referenced from SQL statements /// executed against this context. - pub fn register_avro( + pub async fn register_avro( &mut self, name: &str, - filename: &str, - options: AvroReadOptions, + uri: &str, + options: AvroReadOptions<'_>, ) -> Result<()> { - self.register_table(name, Arc::new(AvroFile::try_new(filename, options)?))?; + let listing_options = options + .to_listing_options(self.state.lock().unwrap().config.target_partitions); + + self.register_listing_table(name, uri, listing_options, options.schema) + .await?; Ok(()) } @@ -417,8 +516,15 @@ impl ExecutionContext { } /// Retrieves a `ObjectStore` instance by scheme - pub fn object_store(&self, scheme: &str) -> Option> { - self.state.lock().unwrap().object_store_registry.get(scheme) + pub fn object_store<'a>( + &self, + uri: &'a str, + ) -> Result<(Arc, &'a str)> { + self.state + .lock() + .unwrap() + .object_store_registry + .get_by_uri(uri) } /// Registers a table using a custom `TableProvider` so that @@ -1176,7 +1282,7 @@ mod tests { async fn create_variable_expr() -> Result<()> { let tmp_dir = TempDir::new()?; let partition_count = 4; - let mut ctx = create_ctx(&tmp_dir, partition_count)?; + let mut ctx = create_ctx(&tmp_dir, partition_count).await?; let variable_provider = test::variable::SystemVar::new(); ctx.register_variable(VarType::System, Arc::new(variable_provider)); @@ -1205,7 +1311,7 @@ mod tests { async fn register_deregister() -> Result<()> { let tmp_dir = TempDir::new()?; let partition_count = 4; - let mut ctx = create_ctx(&tmp_dir, partition_count)?; + let mut ctx = create_ctx(&tmp_dir, partition_count).await?; let provider = test::create_table_dual(); ctx.register_table("dual", provider)?; @@ -1220,7 +1326,7 @@ mod tests { async fn parallel_query_with_filter() -> Result<()> { let tmp_dir = TempDir::new()?; let partition_count = 4; - let ctx = create_ctx(&tmp_dir, partition_count)?; + let ctx = create_ctx(&tmp_dir, partition_count).await?; let logical_plan = ctx.create_logical_plan("SELECT c1, c2 FROM test WHERE c1 > 0 AND c1 < 3")?; @@ -1275,7 +1381,7 @@ mod tests { async fn projection_on_table_scan() -> Result<()> { let tmp_dir = TempDir::new()?; let partition_count = 4; - let ctx = create_ctx(&tmp_dir, partition_count)?; + let ctx = create_ctx(&tmp_dir, partition_count).await?; let table = ctx.table("test")?; let logical_plan = LogicalPlanBuilder::from(table.to_logical_plan()) @@ -1316,7 +1422,7 @@ mod tests { #[tokio::test] async fn preserve_nullability_on_projection() -> Result<()> { let tmp_dir = TempDir::new()?; - let ctx = create_ctx(&tmp_dir, 1)?; + let ctx = create_ctx(&tmp_dir, 1).await?; let schema: Schema = ctx.table("test").unwrap().schema().clone().into(); assert!(!schema.field_with_name("c1")?.is_nullable()); @@ -1910,7 +2016,7 @@ mod tests { #[tokio::test] async fn aggregate_timestamps_sum() -> Result<()> { let tmp_dir = TempDir::new()?; - let mut ctx = create_ctx(&tmp_dir, 1)?; + let mut ctx = create_ctx(&tmp_dir, 1).await?; ctx.register_table("t", test::table_with_timestamps()) .unwrap(); @@ -1929,7 +2035,7 @@ mod tests { #[tokio::test] async fn aggregate_timestamps_count() -> Result<()> { let tmp_dir = TempDir::new()?; - let mut ctx = create_ctx(&tmp_dir, 1)?; + let mut ctx = create_ctx(&tmp_dir, 1).await?; ctx.register_table("t", test::table_with_timestamps()) .unwrap(); @@ -1955,7 +2061,7 @@ mod tests { #[tokio::test] async fn aggregate_timestamps_min() -> Result<()> { let tmp_dir = TempDir::new()?; - let mut ctx = create_ctx(&tmp_dir, 1)?; + let mut ctx = create_ctx(&tmp_dir, 1).await?; ctx.register_table("t", test::table_with_timestamps()) .unwrap(); @@ -1981,7 +2087,7 @@ mod tests { #[tokio::test] async fn aggregate_timestamps_max() -> Result<()> { let tmp_dir = TempDir::new()?; - let mut ctx = create_ctx(&tmp_dir, 1)?; + let mut ctx = create_ctx(&tmp_dir, 1).await?; ctx.register_table("t", test::table_with_timestamps()) .unwrap(); @@ -2007,7 +2113,7 @@ mod tests { #[tokio::test] async fn aggregate_timestamps_avg() -> Result<()> { let tmp_dir = TempDir::new()?; - let mut ctx = create_ctx(&tmp_dir, 1)?; + let mut ctx = create_ctx(&tmp_dir, 1).await?; ctx.register_table("t", test::table_with_timestamps()) .unwrap(); @@ -2063,7 +2169,7 @@ mod tests { #[tokio::test] async fn join_timestamp() -> Result<()> { let tmp_dir = TempDir::new()?; - let mut ctx = create_ctx(&tmp_dir, 1)?; + let mut ctx = create_ctx(&tmp_dir, 1).await?; ctx.register_table("t", test::table_with_timestamps()) .unwrap(); @@ -2190,7 +2296,8 @@ mod tests { "test", tmp_dir.path().to_str().unwrap(), CsvReadOptions::new().schema(&schema).has_header(false), - )?; + ) + .await?; let results = plan_and_collect( &mut ctx, @@ -2420,7 +2527,8 @@ mod tests { "test", tmp_dir.path().to_str().unwrap(), CsvReadOptions::new().schema(&schema).has_header(false), - )?; + ) + .await?; let results = plan_and_collect( &mut ctx, @@ -2507,7 +2615,7 @@ mod tests { #[tokio::test] async fn aggregate_with_alias() -> Result<()> { let tmp_dir = TempDir::new()?; - let ctx = create_ctx(&tmp_dir, 1)?; + let ctx = create_ctx(&tmp_dir, 1).await?; let schema = Arc::new(Schema::new(vec![ Field::new("c1", DataType::Utf8, false), @@ -2533,7 +2641,7 @@ mod tests { #[tokio::test] async fn limit() -> Result<()> { let tmp_dir = TempDir::new()?; - let mut ctx = create_ctx(&tmp_dir, 1)?; + let mut ctx = create_ctx(&tmp_dir, 1).await?; ctx.register_table("t", test::table_with_sequence(1, 1000).unwrap()) .unwrap(); @@ -2573,7 +2681,7 @@ mod tests { #[tokio::test] async fn limit_multi_partitions() -> Result<()> { let tmp_dir = TempDir::new()?; - let mut ctx = create_ctx(&tmp_dir, 1)?; + let mut ctx = create_ctx(&tmp_dir, 1).await?; let partitions = vec![ vec![test::make_partition(0)], @@ -2841,7 +2949,7 @@ mod tests { async fn write_csv_results() -> Result<()> { // create partitioned input file and context let tmp_dir = TempDir::new()?; - let mut ctx = create_ctx(&tmp_dir, 4)?; + let mut ctx = create_ctx(&tmp_dir, 4).await?; // execute a simple query and write the results to CSV let out_dir = tmp_dir.as_ref().to_str().unwrap().to_string() + "/out"; @@ -2857,8 +2965,10 @@ mod tests { // register each partition as well as the top level dir let csv_read_option = CsvReadOptions::new().schema(&schema); - ctx.register_csv("part0", &format!("{}/part-0.csv", out_dir), csv_read_option)?; - ctx.register_csv("allparts", &out_dir, csv_read_option)?; + ctx.register_csv("part0", &format!("{}/part-0.csv", out_dir), csv_read_option) + .await?; + ctx.register_csv("allparts", &out_dir, csv_read_option) + .await?; let part0 = plan_and_collect(&mut ctx, "SELECT c1, c2 FROM part0").await?; let allparts = plan_and_collect(&mut ctx, "SELECT c1, c2 FROM allparts").await?; @@ -2876,7 +2986,7 @@ mod tests { async fn write_parquet_results() -> Result<()> { // create partitioned input file and context let tmp_dir = TempDir::new()?; - let mut ctx = create_ctx(&tmp_dir, 4)?; + let mut ctx = create_ctx(&tmp_dir, 4).await?; // execute a simple query and write the results to CSV let out_dir = tmp_dir.as_ref().to_str().unwrap().to_string() + "/out"; @@ -2886,11 +2996,15 @@ mod tests { let mut ctx = ExecutionContext::new(); // register each partition as well as the top level dir - ctx.register_parquet("part0", &format!("{}/part-0.parquet", out_dir))?; - ctx.register_parquet("part1", &format!("{}/part-1.parquet", out_dir))?; - ctx.register_parquet("part2", &format!("{}/part-2.parquet", out_dir))?; - ctx.register_parquet("part3", &format!("{}/part-3.parquet", out_dir))?; - ctx.register_parquet("allparts", &out_dir)?; + ctx.register_parquet("part0", &format!("{}/part-0.parquet", out_dir)) + .await?; + ctx.register_parquet("part1", &format!("{}/part-1.parquet", out_dir)) + .await?; + ctx.register_parquet("part2", &format!("{}/part-2.parquet", out_dir)) + .await?; + ctx.register_parquet("part3", &format!("{}/part-3.parquet", out_dir)) + .await?; + ctx.register_parquet("allparts", &out_dir).await?; let part0 = plan_and_collect(&mut ctx, "SELECT c1, c2 FROM part0").await?; let allparts = plan_and_collect(&mut ctx, "SELECT c1, c2 FROM allparts").await?; @@ -2919,7 +3033,8 @@ mod tests { CsvReadOptions::new() .schema(&schema) .file_extension(file_extension), - )?; + ) + .await?; let results = plan_and_collect(&mut ctx, "SELECT SUM(c1), SUM(c2), COUNT(*) FROM test") .await?; @@ -2943,7 +3058,7 @@ mod tests { // environment. Usecase is for concurrent planing. let tmp_dir = TempDir::new()?; let partition_count = 4; - let ctx = Arc::new(Mutex::new(create_ctx(&tmp_dir, partition_count)?)); + let ctx = Arc::new(Mutex::new(create_ctx(&tmp_dir, partition_count).await?)); let threads: Vec>> = (0..2) .map(|_| ctx.clone()) @@ -2972,7 +3087,9 @@ mod tests { let opt_plan1 = ctx.optimize(&plan1)?; - let plan2 = ctx.sql("SELECT * FROM (SELECT 1) AS one WHERE TRUE AND TRUE")?; + let plan2 = ctx + .sql("SELECT * FROM (SELECT 1) AS one WHERE TRUE AND TRUE") + .await?; assert_eq!( format!("{:?}", opt_plan1), @@ -3174,7 +3291,7 @@ mod tests { ExecutionConfig::new().with_query_planner(Arc::new(MyQueryPlanner {})), ); - let df = ctx.sql("SELECT 1")?; + let df = ctx.sql("SELECT 1").await?; df.collect().await.expect_err("query not supported"); Ok(()) } @@ -3662,7 +3779,7 @@ mod tests { )); assert!(matches!( - ctx.sql("select * from datafusion.public.test"), + ctx.sql("select * from datafusion.public.test").await, Err(DataFusionError::Plan(_)) )); @@ -3876,6 +3993,7 @@ mod tests { let mut ctx = ExecutionContext::new(); let df = ctx .read_parquet(table_dir.to_str().unwrap().to_string()) + .await .unwrap(); let result = df.collect().await.unwrap(); @@ -3928,13 +4046,13 @@ mod tests { ctx: &mut ExecutionContext, sql: &str, ) -> Result> { - ctx.sql(sql)?.collect().await + ctx.sql(sql).await?.collect().await } /// Execute SQL and return results async fn execute(sql: &str, partition_count: usize) -> Result> { let tmp_dir = TempDir::new()?; - let mut ctx = create_ctx(&tmp_dir, partition_count)?; + let mut ctx = create_ctx(&tmp_dir, partition_count).await?; plan_and_collect(&mut ctx, sql).await } @@ -3994,7 +4112,10 @@ mod tests { } /// Generate a partitioned CSV file and register it with an execution context - fn create_ctx(tmp_dir: &TempDir, partition_count: usize) -> Result { + async fn create_ctx( + tmp_dir: &TempDir, + partition_count: usize, + ) -> Result { let mut ctx = ExecutionContext::with_config( ExecutionConfig::new().with_target_partitions(8), ); @@ -4006,7 +4127,8 @@ mod tests { "test", tmp_dir.path().to_str().unwrap(), CsvReadOptions::new().schema(&schema), - )?; + ) + .await?; Ok(ctx) } diff --git a/datafusion/src/execution/dataframe_impl.rs b/datafusion/src/execution/dataframe_impl.rs index 9971955a6db0..18a558ef7114 100644 --- a/datafusion/src/execution/dataframe_impl.rs +++ b/datafusion/src/execution/dataframe_impl.rs @@ -230,22 +230,23 @@ mod tests { use std::vec; use super::*; + use crate::execution::options::CsvReadOptions; use crate::logical_plan::*; use crate::physical_plan::functions::Volatility; + use crate::physical_plan::ColumnarValue; use crate::{assert_batches_sorted_eq, execution::context::ExecutionContext}; - use crate::{datasource::csv::CsvReadOptions, physical_plan::ColumnarValue}; use crate::{physical_plan::functions::ScalarFunctionImplementation, test}; use arrow::datatypes::DataType; - #[test] - fn select_columns() -> Result<()> { + #[tokio::test] + async fn select_columns() -> Result<()> { // build plan using Table API - let t = test_table()?; + let t = test_table().await?; let t2 = t.select_columns(&["c1", "c2", "c11"])?; let plan = t2.to_logical_plan(); // build query using SQL - let sql_plan = create_plan("SELECT c1, c2, c11 FROM aggregate_test_100")?; + let sql_plan = create_plan("SELECT c1, c2, c11 FROM aggregate_test_100").await?; // the two plans should be identical assert_same_plan(&plan, &sql_plan); @@ -253,15 +254,15 @@ mod tests { Ok(()) } - #[test] - fn select_expr() -> Result<()> { + #[tokio::test] + async fn select_expr() -> Result<()> { // build plan using Table API - let t = test_table()?; + let t = test_table().await?; let t2 = t.select(vec![col("c1"), col("c2"), col("c11")])?; let plan = t2.to_logical_plan(); // build query using SQL - let sql_plan = create_plan("SELECT c1, c2, c11 FROM aggregate_test_100")?; + let sql_plan = create_plan("SELECT c1, c2, c11 FROM aggregate_test_100").await?; // the two plans should be identical assert_same_plan(&plan, &sql_plan); @@ -272,7 +273,7 @@ mod tests { #[tokio::test] async fn aggregate() -> Result<()> { // build plan using DataFrame API - let df = test_table()?; + let df = test_table().await?; let group_expr = vec![col("c1")]; let aggr_expr = vec![ min(col("c12")), @@ -305,8 +306,10 @@ mod tests { #[tokio::test] async fn join() -> Result<()> { - let left = test_table()?.select_columns(&["c1", "c2"])?; - let right = test_table_with_name("c2")?.select_columns(&["c1", "c3"])?; + let left = test_table().await?.select_columns(&["c1", "c2"])?; + let right = test_table_with_name("c2") + .await? + .select_columns(&["c1", "c3"])?; let left_rows = left.collect().await?; let right_rows = right.collect().await?; let join = left.join(right, JoinType::Inner, &["c1"], &["c1"])?; @@ -317,16 +320,16 @@ mod tests { Ok(()) } - #[test] - fn limit() -> Result<()> { + #[tokio::test] + async fn limit() -> Result<()> { // build query using Table API - let t = test_table()?; + let t = test_table().await?; let t2 = t.select_columns(&["c1", "c2", "c11"])?.limit(10)?; let plan = t2.to_logical_plan(); // build query using SQL let sql_plan = - create_plan("SELECT c1, c2, c11 FROM aggregate_test_100 LIMIT 10")?; + create_plan("SELECT c1, c2, c11 FROM aggregate_test_100 LIMIT 10").await?; // the two plans should be identical assert_same_plan(&plan, &sql_plan); @@ -334,10 +337,10 @@ mod tests { Ok(()) } - #[test] - fn explain() -> Result<()> { + #[tokio::test] + async fn explain() -> Result<()> { // build query using Table API - let df = test_table()?; + let df = test_table().await?; let df = df .select_columns(&["c1", "c2", "c11"])? .limit(10)? @@ -346,7 +349,8 @@ mod tests { // build query using SQL let sql_plan = - create_plan("EXPLAIN SELECT c1, c2, c11 FROM aggregate_test_100 LIMIT 10")?; + create_plan("EXPLAIN SELECT c1, c2, c11 FROM aggregate_test_100 LIMIT 10") + .await?; // the two plans should be identical assert_same_plan(&plan, &sql_plan); @@ -354,10 +358,10 @@ mod tests { Ok(()) } - #[test] - fn registry() -> Result<()> { + #[tokio::test] + async fn registry() -> Result<()> { let mut ctx = ExecutionContext::new(); - register_aggregate_csv(&mut ctx, "aggregate_test_100")?; + register_aggregate_csv(&mut ctx, "aggregate_test_100").await?; // declare the udf let my_fn: ScalarFunctionImplementation = @@ -392,7 +396,7 @@ mod tests { #[tokio::test] async fn sendable() { - let df = test_table().unwrap(); + let df = test_table().await.unwrap(); // dataframes should be sendable between threads/tasks let task = tokio::task::spawn(async move { df.select_columns(&["c1"]) @@ -407,23 +411,23 @@ mod tests { } /// Create a logical plan from a SQL query - fn create_plan(sql: &str) -> Result { + async fn create_plan(sql: &str) -> Result { let mut ctx = ExecutionContext::new(); - register_aggregate_csv(&mut ctx, "aggregate_test_100")?; + register_aggregate_csv(&mut ctx, "aggregate_test_100").await?; ctx.create_logical_plan(sql) } - fn test_table_with_name(name: &str) -> Result> { + async fn test_table_with_name(name: &str) -> Result> { let mut ctx = ExecutionContext::new(); - register_aggregate_csv(&mut ctx, name)?; + register_aggregate_csv(&mut ctx, name).await?; ctx.table(name) } - fn test_table() -> Result> { - test_table_with_name("aggregate_test_100") + async fn test_table() -> Result> { + test_table_with_name("aggregate_test_100").await } - fn register_aggregate_csv( + async fn register_aggregate_csv( ctx: &mut ExecutionContext, table_name: &str, ) -> Result<()> { @@ -433,7 +437,8 @@ mod tests { table_name, &format!("{}/csv/aggregate_test_100.csv", testdata), CsvReadOptions::new().schema(schema.as_ref()), - )?; + ) + .await?; Ok(()) } } diff --git a/datafusion/src/execution/mod.rs b/datafusion/src/execution/mod.rs index ff44dd43f834..e353a3160b8d 100644 --- a/datafusion/src/execution/mod.rs +++ b/datafusion/src/execution/mod.rs @@ -19,3 +19,4 @@ pub mod context; pub mod dataframe_impl; +pub mod options; diff --git a/datafusion/src/execution/options.rs b/datafusion/src/execution/options.rs new file mode 100644 index 000000000000..f0ed6f24c325 --- /dev/null +++ b/datafusion/src/execution/options.rs @@ -0,0 +1,173 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! User facing options for the file formats readers + +use std::sync::Arc; + +use arrow::datatypes::{Schema, SchemaRef}; + +use crate::datasource::{ + file_format::{avro::AvroFormat, csv::CsvFormat}, + listing::ListingOptions, +}; + +/// CSV file read option +#[derive(Copy, Clone)] +pub struct CsvReadOptions<'a> { + /// Does the CSV file have a header? + /// + /// If schema inference is run on a file with no headers, default column names + /// are created. + pub has_header: bool, + /// An optional column delimiter. Defaults to `b','`. + pub delimiter: u8, + /// An optional schema representing the CSV files. If None, CSV reader will try to infer it + /// based on data in file. + pub schema: Option<&'a Schema>, + /// Max number of rows to read from CSV files for schema inference if needed. Defaults to 1000. + pub schema_infer_max_records: usize, + /// File extension; only files with this extension are selected for data input. + /// Defaults to ".csv". + pub file_extension: &'a str, +} + +impl<'a> CsvReadOptions<'a> { + /// Create a CSV read option with default presets + pub fn new() -> Self { + Self { + has_header: true, + schema: None, + schema_infer_max_records: 1000, + delimiter: b',', + file_extension: ".csv", + } + } + + /// Configure has_header setting + pub fn has_header(mut self, has_header: bool) -> Self { + self.has_header = has_header; + self + } + + /// Specify delimiter to use for CSV read + pub fn delimiter(mut self, delimiter: u8) -> Self { + self.delimiter = delimiter; + self + } + + /// Specify the file extension for CSV file selection + pub fn file_extension(mut self, file_extension: &'a str) -> Self { + self.file_extension = file_extension; + self + } + + /// Configure delimiter setting with Option, None value will be ignored + pub fn delimiter_option(mut self, delimiter: Option) -> Self { + if let Some(d) = delimiter { + self.delimiter = d; + } + self + } + + /// Specify schema to use for CSV read + pub fn schema(mut self, schema: &'a Schema) -> Self { + self.schema = Some(schema); + self + } + + /// Configure number of max records to read for schema inference + pub fn schema_infer_max_records(mut self, max_records: usize) -> Self { + self.schema_infer_max_records = max_records; + self + } + + /// Helper to convert these user facing options to `ListingTable` options + pub fn to_listing_options(&self, target_partitions: usize) -> ListingOptions { + let file_format = CsvFormat::default() + .with_has_header(self.has_header) + .with_delimiter(self.delimiter) + .with_schema_infer_max_rec(Some(self.schema_infer_max_records)); + + ListingOptions { + format: Arc::new(file_format), + collect_stat: false, + file_extension: self.file_extension.to_owned(), + target_partitions, + partitions: vec![], + } + } +} + +/// Avro read options +#[derive(Clone)] +pub struct AvroReadOptions<'a> { + /// The data source schema. + pub schema: Option, + + /// File extension; only files with this extension are selected for data input. + /// Defaults to ".avro". + pub file_extension: &'a str, +} + +impl<'a> Default for AvroReadOptions<'a> { + fn default() -> Self { + Self { + schema: None, + file_extension: ".avro", + } + } +} + +impl<'a> AvroReadOptions<'a> { + /// Helper to convert these user facing options to `ListingTable` options + pub fn to_listing_options(&self, target_partitions: usize) -> ListingOptions { + let file_format = AvroFormat::default(); + + ListingOptions { + format: Arc::new(file_format), + collect_stat: false, + file_extension: self.file_extension.to_owned(), + target_partitions, + partitions: vec![], + } + } +} + +/// Line-delimited JSON read options +#[derive(Clone)] +pub struct NdJsonReadOptions<'a> { + /// The data source schema. + pub schema: Option, + + /// Max number of rows to read from CSV files for schema inference if needed. Defaults to 1000. + pub schema_infer_max_records: usize, + + /// File extension; only files with this extension are selected for data input. + /// Defaults to ".json". + pub file_extension: &'a str, +} + +impl<'a> Default for NdJsonReadOptions<'a> { + fn default() -> Self { + Self { + schema: None, + schema_infer_max_records: 1000, + file_extension: ".json", + } + } +} diff --git a/datafusion/src/lib.rs b/datafusion/src/lib.rs index adaca114d2d6..a4a5a88d16b5 100644 --- a/datafusion/src/lib.rs +++ b/datafusion/src/lib.rs @@ -46,7 +46,7 @@ //! let mut ctx = ExecutionContext::new(); //! //! // create the dataframe -//! let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new())?; +//! let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new()).await?; //! //! // create a plan //! let df = df.filter(col("a").lt_eq(col("b")))? @@ -83,10 +83,10 @@ //! # async fn main() -> Result<()> { //! let mut ctx = ExecutionContext::new(); //! -//! ctx.register_csv("example", "tests/example.csv", CsvReadOptions::new())?; +//! ctx.register_csv("example", "tests/example.csv", CsvReadOptions::new()).await?; //! //! // create a plan -//! let df = ctx.sql("SELECT a, MIN(b) FROM example GROUP BY a LIMIT 100")?; +//! let df = ctx.sql("SELECT a, MIN(b) FROM example GROUP BY a LIMIT 100").await?; //! //! // execute the plan //! let results: Vec = df.collect().await?; diff --git a/datafusion/src/logical_plan/builder.rs b/datafusion/src/logical_plan/builder.rs index d4f941a996d2..3a1d12735658 100644 --- a/datafusion/src/logical_plan/builder.rs +++ b/datafusion/src/logical_plan/builder.rs @@ -27,21 +27,23 @@ use arrow::{ record_batch::RecordBatch, }; -use crate::error::{DataFusionError, Result}; -use crate::{datasource::TableProvider, logical_plan::plan::ToStringifiedPlan}; -use crate::{ - datasource::{empty::EmptyTable, parquet::ParquetTable, CsvFile, MemTable}, - prelude::CsvReadOptions, +use crate::datasource::{ + empty::EmptyTable, + file_format::parquet::{ParquetFormat, DEFAULT_PARQUET_EXTENSION}, + listing::{ListingOptions, ListingTable}, + object_store::ObjectStore, + MemTable, TableProvider, }; +use crate::error::{DataFusionError, Result}; +use crate::logical_plan::plan::ToStringifiedPlan; +use crate::prelude::*; use super::dfschema::ToDFSchema; use super::{exprlist_to_fields, Expr, JoinConstraint, JoinType, LogicalPlan, PlanType}; -use crate::datasource::avro::AvroFile; use crate::logical_plan::{ columnize_expr, normalize_col, normalize_cols, Column, DFField, DFSchema, DFSchemaRef, Partitioning, }; -use crate::physical_plan::avro::AvroReadOptions; /// Default table name for unnamed table pub const UNNAMED_TABLE: &str = "?table?"; @@ -120,66 +122,146 @@ impl LogicalPlanBuilder { } /// Scan a CSV data source - pub fn scan_csv( + pub async fn scan_csv( + object_store: Arc, path: impl Into, - options: CsvReadOptions, + options: CsvReadOptions<'_>, projection: Option>, + target_partitions: usize, ) -> Result { let path = path.into(); - Self::scan_csv_with_name(path.clone(), options, projection, path) + Self::scan_csv_with_name( + object_store, + path.clone(), + options, + projection, + path, + target_partitions, + ) + .await } /// Scan a CSV data source and register it with a given table name - pub fn scan_csv_with_name( + pub async fn scan_csv_with_name( + object_store: Arc, path: impl Into, - options: CsvReadOptions, + options: CsvReadOptions<'_>, projection: Option>, table_name: impl Into, + target_partitions: usize, ) -> Result { - let provider = Arc::new(CsvFile::try_new(path, options)?); - Self::scan(table_name, provider, projection) + let listing_options = options.to_listing_options(target_partitions); + + let path: String = path.into(); + + let resolved_schema = match options.schema { + Some(s) => Arc::new(s.to_owned()), + None => { + listing_options + .infer_schema(Arc::clone(&object_store), &path) + .await? + } + }; + let provider = + ListingTable::new(object_store, path, resolved_schema, listing_options); + + Self::scan(table_name, Arc::new(provider), projection) } /// Scan a Parquet data source - pub fn scan_parquet( + pub async fn scan_parquet( + object_store: Arc, path: impl Into, projection: Option>, target_partitions: usize, ) -> Result { let path = path.into(); - Self::scan_parquet_with_name(path.clone(), projection, target_partitions, path) + Self::scan_parquet_with_name( + object_store, + path.clone(), + projection, + target_partitions, + path, + ) + .await } /// Scan a Parquet data source and register it with a given table name - pub fn scan_parquet_with_name( + pub async fn scan_parquet_with_name( + object_store: Arc, path: impl Into, projection: Option>, target_partitions: usize, table_name: impl Into, ) -> Result { - let provider = Arc::new(ParquetTable::try_new(path, target_partitions)?); - Self::scan(table_name, provider, projection) + // TODO remove hard coded enable_pruning + let file_format = ParquetFormat::default().with_enable_pruning(true); + + let listing_options = ListingOptions { + format: Arc::new(file_format), + collect_stat: true, + file_extension: DEFAULT_PARQUET_EXTENSION.to_owned(), + target_partitions, + partitions: vec![], + }; + + let path: String = path.into(); + + // with parquet we resolve the schema in all cases + let resolved_schema = listing_options + .infer_schema(Arc::clone(&object_store), &path) + .await?; + + let provider = + ListingTable::new(object_store, path, resolved_schema, listing_options); + Self::scan(table_name, Arc::new(provider), projection) } /// Scan an Avro data source - pub fn scan_avro( + pub async fn scan_avro( + object_store: Arc, path: impl Into, - options: AvroReadOptions, + options: AvroReadOptions<'_>, projection: Option>, + target_partitions: usize, ) -> Result { let path = path.into(); - Self::scan_avro_with_name(path.clone(), options, projection, path) + Self::scan_avro_with_name( + object_store, + path.clone(), + options, + projection, + path, + target_partitions, + ) + .await } /// Scan an Avro data source and register it with a given table name - pub fn scan_avro_with_name( + pub async fn scan_avro_with_name( + object_store: Arc, path: impl Into, - options: AvroReadOptions, + options: AvroReadOptions<'_>, projection: Option>, table_name: impl Into, + target_partitions: usize, ) -> Result { - let provider = Arc::new(AvroFile::try_new(&path.into(), options)?); - Self::scan(table_name, provider, projection) + let listing_options = options.to_listing_options(target_partitions); + + let path: String = path.into(); + + let resolved_schema = match options.schema { + Some(s) => s, + None => { + listing_options + .infer_schema(Arc::clone(&object_store), &path) + .await? + } + }; + let provider = + ListingTable::new(object_store, path, resolved_schema, listing_options); + + Self::scan(table_name, Arc::new(provider), projection) } /// Scan an empty data source, mainly used in tests @@ -198,6 +280,16 @@ impl LogicalPlanBuilder { table_name: impl Into, provider: Arc, projection: Option>, + ) -> Result { + Self::scan_with_filters(table_name, provider, projection, vec![]) + } + + /// Convert a table provider into a builder with a TableScan + pub fn scan_with_filters( + table_name: impl Into, + provider: Arc, + projection: Option>, + filters: Vec, ) -> Result { let table_name = table_name.into(); @@ -229,7 +321,7 @@ impl LogicalPlanBuilder { source: provider, projected_schema: Arc::new(projected_schema), projection, - filters: vec![], + filters, limit: None, }; diff --git a/datafusion/src/physical_optimizer/repartition.rs b/datafusion/src/physical_optimizer/repartition.rs index 61266e442c98..af47d86f65bc 100644 --- a/datafusion/src/physical_optimizer/repartition.rs +++ b/datafusion/src/physical_optimizer/repartition.rs @@ -109,28 +109,23 @@ mod tests { use arrow::datatypes::Schema; use super::*; + use crate::datasource::object_store::local::LocalFileSystem; use crate::datasource::PartitionedFile; - use crate::physical_plan::metrics::ExecutionPlanMetricsSet; - use crate::physical_plan::parquet::{ParquetExec, ParquetPartition}; + use crate::physical_plan::file_format::ParquetExec; use crate::physical_plan::projection::ProjectionExec; use crate::physical_plan::Statistics; #[test] fn added_repartition_to_single_partition() -> Result<()> { let schema = Arc::new(Schema::empty()); - let metrics = ExecutionPlanMetricsSet::new(); let parquet_project = ProjectionExec::try_new( vec![], Arc::new(ParquetExec::new( - vec![ParquetPartition::new( - vec![PartitionedFile::from("x".to_string())], - 0, - metrics.clone(), - )], + Arc::new(LocalFileSystem {}), + vec![vec![PartitionedFile::new("x".to_string(), 100)]], + Statistics::default(), schema, None, - Statistics::default(), - metrics, None, 2048, None, @@ -157,21 +152,16 @@ mod tests { #[test] fn repartition_deepest_node() -> Result<()> { let schema = Arc::new(Schema::empty()); - let metrics = ExecutionPlanMetricsSet::new(); let parquet_project = ProjectionExec::try_new( vec![], Arc::new(ProjectionExec::try_new( vec![], Arc::new(ParquetExec::new( - vec![ParquetPartition::new( - vec![PartitionedFile::from("x".to_string())], - 0, - metrics.clone(), - )], + Arc::new(LocalFileSystem {}), + vec![vec![PartitionedFile::new("x".to_string(), 100)]], + Statistics::default(), schema, None, - Statistics::default(), - metrics, None, 2048, None, diff --git a/datafusion/src/physical_plan/avro.rs b/datafusion/src/physical_plan/avro.rs deleted file mode 100644 index 3f0b007b26c0..000000000000 --- a/datafusion/src/physical_plan/avro.rs +++ /dev/null @@ -1,457 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -//! Execution plan for reading line-delimited Avro files -#[cfg(feature = "avro")] -use super::RecordBatchStream; -use super::{common, source::Source, ExecutionPlan, Partitioning}; -use crate::avro_to_arrow::read_avro_schema_from_reader; -use crate::error::{DataFusionError, Result}; -use crate::physical_plan::{DisplayFormatType, Statistics}; -use arrow::datatypes::{Schema, SchemaRef}; -#[cfg(feature = "avro")] -use arrow::{error::Result as ArrowResult, record_batch::RecordBatch}; -use async_trait::async_trait; -#[cfg(feature = "avro")] -use futures::Stream; -use std::fs::File; -use std::{any::Any, io::Seek}; -use std::{ - io::Read, - sync::{Arc, Mutex}, -}; -#[cfg(feature = "avro")] -use std::{ - pin::Pin, - task::{Context, Poll}, -}; - -/// Line-delimited Avro read options -#[derive(Clone)] -pub struct AvroReadOptions<'a> { - /// The data source schema. - pub schema: Option, - - /// File extension; only files with this extension are selected for data input. - /// Defaults to ".avro". - pub file_extension: &'a str, -} - -impl<'a> Default for AvroReadOptions<'a> { - fn default() -> Self { - Self { - schema: None, - file_extension: ".avro", - } - } -} - -trait SeekRead: Read + Seek {} - -impl SeekRead for T {} -/// Execution plan for scanning Avro data source -#[derive(Debug)] -pub struct AvroExec { - source: Source>, - schema: SchemaRef, - projection: Option>, - projected_schema: SchemaRef, - file_extension: String, - batch_size: usize, - limit: Option, -} - -impl AvroExec { - /// Create a new execution plan for reading from a path - pub fn try_from_path( - path: &str, - options: AvroReadOptions, - projection: Option>, - batch_size: usize, - limit: Option, - ) -> Result { - let file_extension = options.file_extension.to_string(); - - let filenames = common::build_file_list(path, &file_extension)?; - - if filenames.is_empty() { - return Err(DataFusionError::Execution(format!( - "No files found at {path} with file extension {file_extension}", - path = path, - file_extension = file_extension.as_str() - ))); - } - - let schema = match options.schema { - Some(s) => s, - None => Arc::new(AvroExec::try_read_schema(filenames.as_slice())?), - }; - - let projected_schema = match &projection { - None => schema.clone(), - Some(p) => Arc::new(Schema::new( - p.iter().map(|i| schema.field(*i).clone()).collect(), - )), - }; - - Ok(Self { - source: Source::PartitionedFiles { - path: path.to_string(), - filenames, - }, - schema, - projected_schema, - file_extension, - projection, - batch_size, - limit, - }) - } - /// Create a new execution plan for reading from a reader - pub fn try_new_from_reader( - reader: impl Read + Seek + Send + Sync + 'static, - options: AvroReadOptions, - projection: Option>, - batch_size: usize, - limit: Option, - ) -> Result { - let schema = match options.schema { - Some(s) => s, - None => { - return Err(DataFusionError::Execution( - "The schema must be provided in options when reading from a reader" - .to_string(), - )); - } - }; - - let projected_schema = match &projection { - None => schema.clone(), - Some(p) => Arc::new(Schema::new( - p.iter().map(|i| schema.field(*i).clone()).collect(), - )), - }; - - Ok(Self { - source: Source::Reader(Mutex::new(Some(Box::new(reader)))), - schema, - file_extension: String::new(), - projection, - projected_schema, - batch_size, - limit, - }) - } - - /// Path to directory containing partitioned CSV files with the same schema - pub fn path(&self) -> &str { - self.source.path() - } - - /// The individual files under path - pub fn filenames(&self) -> &[String] { - self.source.filenames() - } - - /// File extension - pub fn file_extension(&self) -> &str { - &self.file_extension - } - - /// Get the schema of the avro file - pub fn file_schema(&self) -> SchemaRef { - self.schema.clone() - } - - /// Optional projection for which columns to load - pub fn projection(&self) -> Option<&Vec> { - self.projection.as_ref() - } - - /// Batch size - pub fn batch_size(&self) -> usize { - self.batch_size - } - - /// Limit - pub fn limit(&self) -> Option { - self.limit - } - - /// Read schema for given Avro dataset - pub fn try_read_schema(filenames: &[String]) -> Result { - let mut schemas = Vec::new(); - for filename in filenames { - let mut file = File::open(filename)?; - let schema = read_avro_schema_from_reader(&mut file)?; - schemas.push(schema); - } - - Ok(Schema::try_merge(schemas)?) - } -} - -#[async_trait] -impl ExecutionPlan for AvroExec { - fn as_any(&self) -> &dyn Any { - self - } - - fn schema(&self) -> SchemaRef { - self.projected_schema.clone() - } - - fn output_partitioning(&self) -> Partitioning { - Partitioning::UnknownPartitioning(match &self.source { - Source::PartitionedFiles { filenames, .. } => filenames.len(), - Source::Reader(_) => 1, - }) - } - - fn children(&self) -> Vec> { - Vec::new() - } - - fn with_new_children( - &self, - children: Vec>, - ) -> Result> { - if !children.is_empty() { - Err(DataFusionError::Internal(format!( - "Children cannot be replaced in {:?}", - self - ))) - } else if let Source::PartitionedFiles { filenames, path } = &self.source { - Ok(Arc::new(Self { - source: Source::PartitionedFiles { - filenames: filenames.clone(), - path: path.clone(), - }, - schema: self.schema.clone(), - projection: self.projection.clone(), - projected_schema: self.projected_schema.clone(), - batch_size: self.batch_size, - limit: self.limit, - file_extension: self.file_extension.clone(), - })) - } else { - Err(DataFusionError::Internal( - "AvroExec with reader source cannot be used with `with_new_children`" - .to_string(), - )) - } - } - - #[cfg(not(feature = "avro"))] - async fn execute( - &self, - _partition: usize, - ) -> Result { - Err(DataFusionError::NotImplemented( - "Cannot execute avro plan without avro feature enabled".to_string(), - )) - } - - #[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(), - ); - } - match &self.source { - Source::PartitionedFiles { filenames, .. } => { - let file = File::open(&filenames[partition])?; - - Ok(Box::pin(AvroStream::new(builder.build(file)?, self.limit))) - } - Source::Reader(rdr) => { - if partition != 0 { - Err(DataFusionError::Internal( - "Only partition 0 is valid when Avro comes from a reader" - .to_string(), - )) - } else if let Some(rdr) = rdr.lock().unwrap().take() { - Ok(Box::pin(AvroStream::new(builder.build(rdr)?, self.limit))) - } else { - Err(DataFusionError::Execution( - "Error reading Avro: Data can only be read a single time when the source is a reader" - .to_string(), - )) - } - } - } - } - - fn fmt_as( - &self, - t: DisplayFormatType, - f: &mut std::fmt::Formatter, - ) -> std::fmt::Result { - match t { - DisplayFormatType::Default => { - write!( - f, - "AvroExec: source={}, batch_size={}, limit={:?}", - self.source, self.batch_size, self.limit - ) - } - } - } - - fn statistics(&self) -> Statistics { - Statistics::default() - } -} - -#[cfg(feature = "avro")] -struct AvroStream<'a, R: Read> { - reader: crate::avro_to_arrow::Reader<'a, R>, - remain: Option, -} - -#[cfg(feature = "avro")] -impl<'a, R: Read> AvroStream<'a, R> { - fn new(reader: crate::avro_to_arrow::Reader<'a, R>, limit: Option) -> Self { - Self { - reader, - remain: limit, - } - } -} - -#[cfg(feature = "avro")] -impl Stream for AvroStream<'_, R> { - type Item = ArrowResult; - - fn poll_next( - mut self: Pin<&mut Self>, - _cx: &mut Context<'_>, - ) -> Poll> { - if let Some(remain) = self.remain.as_mut() { - if *remain < 1 { - return Poll::Ready(None); - } - } - - Poll::Ready(match self.reader.next() { - Ok(Some(item)) => { - if let Some(remain) = self.remain.as_mut() { - if *remain >= item.num_rows() { - *remain -= item.num_rows(); - Some(Ok(item)) - } else { - let len = *remain; - *remain = 0; - Some(Ok(RecordBatch::try_new( - item.schema(), - item.columns() - .iter() - .map(|column| column.slice(0, len)) - .collect(), - )?)) - } - } else { - Some(Ok(item)) - } - } - Ok(None) => None, - Err(err) => Some(Err(err)), - }) - } -} - -#[cfg(feature = "avro")] -impl RecordBatchStream for AvroStream<'_, R> { - fn schema(&self) -> SchemaRef { - self.reader.schema() - } -} - -#[cfg(test)] -mod tests { - use super::*; - - #[tokio::test] - #[cfg(feature = "avro")] - async fn test() -> Result<()> { - use futures::StreamExt; - - let testdata = crate::test_util::arrow_test_data(); - let filename = format!("{}/avro/alltypes_plain.avro", testdata); - let avro_exec = AvroExec::try_from_path( - &filename, - AvroReadOptions::default(), - Some(vec![0, 1, 2]), - 1024, - None, - )?; - assert_eq!(avro_exec.output_partitioning().partition_count(), 1); - - let mut results = avro_exec.execute(0).await?; - let batch = results.next().await.unwrap()?; - - assert_eq!(8, batch.num_rows()); - assert_eq!(3, batch.num_columns()); - - let schema = batch.schema(); - let field_names: Vec<&str> = - schema.fields().iter().map(|f| f.name().as_str()).collect(); - assert_eq!(vec!["id", "bool_col", "tinyint_col"], field_names); - - let batch = results.next().await; - assert!(batch.is_none()); - - let batch = results.next().await; - assert!(batch.is_none()); - - let batch = results.next().await; - assert!(batch.is_none()); - - Ok(()) - } - - #[tokio::test] - #[cfg(not(feature = "avro"))] - async fn test() -> Result<()> { - let testdata = crate::test_util::arrow_test_data(); - let filename = format!("{}/avro/alltypes_plain.avro", testdata); - let avro_exec = AvroExec::try_from_path( - &filename, - AvroReadOptions::default(), - Some(vec![0, 1, 2]), - 1024, - None, - ); - assert!(matches!( - avro_exec, - Err(DataFusionError::NotImplemented(msg)) - if msg == *"cannot read avro schema without the 'avro' feature enabled" - )); - - Ok(()) - } -} diff --git a/datafusion/src/physical_plan/coalesce_partitions.rs b/datafusion/src/physical_plan/coalesce_partitions.rs index 329edcb31d65..a1068386f0d2 100644 --- a/datafusion/src/physical_plan/coalesce_partitions.rs +++ b/datafusion/src/physical_plan/coalesce_partitions.rs @@ -195,8 +195,9 @@ impl RecordBatchStream for MergeStream { mod tests { use super::*; + use crate::datasource::object_store::local::LocalFileSystem; use crate::physical_plan::common; - use crate::physical_plan::csv::{CsvExec, CsvReadOptions}; + use crate::physical_plan::file_format::CsvExec; use crate::test; #[tokio::test] @@ -204,16 +205,19 @@ mod tests { let schema = test::aggr_test_schema(); let num_partitions = 4; - let path = + let (_, files) = test::create_partitioned_csv("aggregate_test_100.csv", num_partitions)?; - - let csv = CsvExec::try_new( - &path, - CsvReadOptions::new().schema(&schema), + let csv = CsvExec::new( + Arc::new(LocalFileSystem {}), + files, + Statistics::default(), + schema, + true, + b',', None, 1024, None, - )?; + ); // input should have 4 partitions assert_eq!(csv.output_partitioning().partition_count(), num_partitions); diff --git a/datafusion/src/physical_plan/csv.rs b/datafusion/src/physical_plan/csv.rs deleted file mode 100644 index 35bd2247bfbc..000000000000 --- a/datafusion/src/physical_plan/csv.rs +++ /dev/null @@ -1,534 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -//! Execution plan for reading CSV files - -use crate::error::{DataFusionError, Result}; -use crate::physical_plan::ExecutionPlan; -use crate::physical_plan::{common, source::Source, Partitioning}; -use arrow::csv; -use arrow::datatypes::{Schema, SchemaRef}; -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; -use std::sync::Mutex; -use std::task::{Context, Poll}; - -use super::{ - DisplayFormatType, RecordBatchStream, SendableRecordBatchStream, Statistics, -}; -use async_trait::async_trait; - -/// CSV file read option -#[derive(Copy, Clone)] -pub struct CsvReadOptions<'a> { - /// Does the CSV file have a header? - /// - /// If schema inference is run on a file with no headers, default column names - /// are created. - pub has_header: bool, - /// An optional column delimiter. Defaults to `b','`. - pub delimiter: u8, - /// An optional schema representing the CSV files. If None, CSV reader will try to infer it - /// based on data in file. - pub schema: Option<&'a Schema>, - /// Max number of rows to read from CSV files for schema inference if needed. Defaults to 1000. - pub schema_infer_max_records: usize, - /// File extension; only files with this extension are selected for data input. - /// Defaults to ".csv". - pub file_extension: &'a str, -} - -impl<'a> CsvReadOptions<'a> { - /// Create a CSV read option with default presets - pub fn new() -> Self { - Self { - has_header: true, - schema: None, - schema_infer_max_records: 1000, - delimiter: b',', - file_extension: ".csv", - } - } - - /// Configure has_header setting - pub fn has_header(mut self, has_header: bool) -> Self { - self.has_header = has_header; - self - } - - /// Specify delimiter to use for CSV read - pub fn delimiter(mut self, delimiter: u8) -> Self { - self.delimiter = delimiter; - self - } - - /// Specify the file extension for CSV file selection - pub fn file_extension(mut self, file_extension: &'a str) -> Self { - self.file_extension = file_extension; - self - } - - /// Configure delimiter setting with Option, None value will be ignored - pub fn delimiter_option(mut self, delimiter: Option) -> Self { - if let Some(d) = delimiter { - self.delimiter = d; - } - self - } - - /// Specify schema to use for CSV read - pub fn schema(mut self, schema: &'a Schema) -> Self { - self.schema = Some(schema); - self - } - - /// Configure number of max records to read for schema inference - pub fn schema_infer_max_records(mut self, max_records: usize) -> Self { - self.schema_infer_max_records = max_records; - self - } -} - -/// Execution plan for scanning a CSV file -#[derive(Debug, Clone)] -pub struct CsvExec { - /// Where the data comes from. - source: Source, - /// Schema representing the CSV file - schema: SchemaRef, - /// Does the CSV file have a header? - has_header: bool, - /// An optional column delimiter. Defaults to `b','` - delimiter: Option, - /// File extension - file_extension: String, - /// Optional projection for which columns to load - projection: Option>, - /// Schema after the projection has been applied - projected_schema: SchemaRef, - /// Batch size - batch_size: usize, - /// Limit in nr. of rows - limit: Option, -} - -impl CsvExec { - /// Create a new execution plan for reading a set of CSV files - pub fn try_new( - path: &str, - options: CsvReadOptions, - projection: Option>, - batch_size: usize, - limit: Option, - ) -> Result { - let file_extension = String::from(options.file_extension); - - let filenames = common::build_file_list(path, file_extension.as_str())?; - if filenames.is_empty() { - return Err(DataFusionError::Execution(format!( - "No files found at {path} with file extension {file_extension}", - path = path, - file_extension = file_extension.as_str() - ))); - } - - let schema = match options.schema { - Some(s) => s.clone(), - None => CsvExec::try_infer_schema(&filenames, &options)?, - }; - - let projected_schema = match &projection { - None => schema.clone(), - Some(p) => Schema::new(p.iter().map(|i| schema.field(*i).clone()).collect()), - }; - - Ok(Self { - source: Source::PartitionedFiles { - path: path.to_string(), - filenames, - }, - schema: Arc::new(schema), - has_header: options.has_header, - delimiter: Some(options.delimiter), - file_extension, - projection, - projected_schema: Arc::new(projected_schema), - batch_size, - limit, - }) - } - /// Create a new execution plan for reading from a reader - pub fn try_new_from_reader( - reader: impl Read + Send + Sync + 'static, - options: CsvReadOptions, - projection: Option>, - batch_size: usize, - limit: Option, - ) -> Result { - let schema = match options.schema { - Some(s) => s.clone(), - None => { - return Err(DataFusionError::Execution( - "The schema must be provided in options when reading from a reader" - .to_string(), - )); - } - }; - - let projected_schema = match &projection { - None => schema.clone(), - Some(p) => Schema::new(p.iter().map(|i| schema.field(*i).clone()).collect()), - }; - - Ok(Self { - source: Source::Reader(Mutex::new(Some(Box::new(reader)))), - schema: Arc::new(schema), - has_header: options.has_header, - delimiter: Some(options.delimiter), - file_extension: String::new(), - projection, - projected_schema: Arc::new(projected_schema), - batch_size, - limit, - }) - } - - /// Path to directory containing partitioned CSV files with the same schema - pub fn path(&self) -> &str { - self.source.path() - } - - /// The individual files under path - pub fn filenames(&self) -> &[String] { - self.source.filenames() - } - - /// Does the CSV file have a header? - pub fn has_header(&self) -> bool { - self.has_header - } - - /// An optional column delimiter. Defaults to `b','` - pub fn delimiter(&self) -> Option<&u8> { - self.delimiter.as_ref() - } - - /// File extension - pub fn file_extension(&self) -> &str { - &self.file_extension - } - - /// Get the schema of the CSV file - pub fn file_schema(&self) -> SchemaRef { - self.schema.clone() - } - - /// Optional projection for which columns to load - pub fn projection(&self) -> Option<&Vec> { - self.projection.as_ref() - } - - /// Batch size - pub fn batch_size(&self) -> usize { - self.batch_size - } - - /// Limit - pub fn limit(&self) -> Option { - self.limit - } - - /// Infer schema for given CSV dataset - pub fn try_infer_schema( - filenames: &[String], - options: &CsvReadOptions, - ) -> Result { - Ok(csv::infer_schema_from_files( - filenames, - options.delimiter, - Some(options.schema_infer_max_records), - options.has_header, - )?) - } -} - -#[async_trait] -impl ExecutionPlan for CsvExec { - /// Return a reference to Any that can be used for downcasting - fn as_any(&self) -> &dyn Any { - self - } - - /// Get the schema for this execution plan - fn schema(&self) -> SchemaRef { - self.projected_schema.clone() - } - - /// Get the output partitioning of this plan - fn output_partitioning(&self) -> Partitioning { - Partitioning::UnknownPartitioning(match &self.source { - Source::PartitionedFiles { filenames, .. } => filenames.len(), - Source::Reader(_) => 1, - }) - } - - fn children(&self) -> Vec> { - // this is a leaf node and has no children - vec![] - } - - fn with_new_children( - &self, - children: Vec>, - ) -> Result> { - if children.is_empty() { - Ok(Arc::new(self.clone())) - } else { - Err(DataFusionError::Internal(format!( - "Children cannot be replaced in {:?}", - self - ))) - } - } - - async fn execute(&self, partition: usize) -> Result { - match &self.source { - Source::PartitionedFiles { filenames, .. } => { - Ok(Box::pin(CsvStream::try_new( - &filenames[partition], - self.schema.clone(), - self.has_header, - self.delimiter, - &self.projection, - self.batch_size, - self.limit, - )?)) - } - Source::Reader(rdr) => { - if partition != 0 { - Err(DataFusionError::Internal( - "Only partition 0 is valid when CSV comes from a reader" - .to_string(), - )) - } else if let Some(rdr) = rdr.lock().unwrap().take() { - Ok(Box::pin(CsvStream::try_new_from_reader( - rdr, - self.schema.clone(), - self.has_header, - self.delimiter, - &self.projection, - self.batch_size, - self.limit, - )?)) - } else { - Err(DataFusionError::Execution( - "Error reading CSV: Data can only be read a single time when the source is a reader" - .to_string(), - )) - } - } - } - } - - fn fmt_as( - &self, - t: DisplayFormatType, - f: &mut std::fmt::Formatter, - ) -> std::fmt::Result { - match t { - DisplayFormatType::Default => { - write!( - f, - "CsvExec: source={}, has_header={}", - self.source, self.has_header - ) - } - } - } - - fn statistics(&self) -> Statistics { - // TODO stats: handle statistics - Statistics::default() - } -} - -/// Iterator over batches -struct CsvStream { - /// Arrow CSV reader - reader: csv::Reader, -} -impl CsvStream { - /// Create an iterator for a CSV file - pub fn try_new( - filename: &str, - schema: SchemaRef, - has_header: bool, - delimiter: Option, - projection: &Option>, - batch_size: usize, - limit: Option, - ) -> Result { - let file = File::open(filename)?; - 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( - reader: R, - schema: SchemaRef, - has_header: bool, - delimiter: Option, - projection: &Option>, - batch_size: usize, - limit: Option, - ) -> Result> { - let start_line = if has_header { 1 } else { 0 }; - let bounds = limit.map(|x| (0, x + start_line)); - - let reader = csv::Reader::new( - reader, - schema, - has_header, - delimiter, - batch_size, - bounds, - projection.clone(), - ); - - Ok(Self { reader }) - } -} - -impl Stream for CsvStream { - type Item = ArrowResult; - - fn poll_next( - mut self: Pin<&mut Self>, - _: &mut Context<'_>, - ) -> Poll> { - Poll::Ready(self.reader.next()) - } -} - -impl RecordBatchStream for CsvStream { - /// Get the schema - fn schema(&self) -> SchemaRef { - self.reader.schema() - } -} - -#[cfg(test)] -mod tests { - use super::*; - use crate::test::aggr_test_schema; - use futures::StreamExt; - - #[tokio::test] - async fn csv_exec_with_projection() -> Result<()> { - let schema = aggr_test_schema(); - let testdata = crate::test_util::arrow_test_data(); - let filename = "aggregate_test_100.csv"; - let path = format!("{}/csv/{}", testdata, filename); - let csv = CsvExec::try_new( - &path, - CsvReadOptions::new().schema(&schema), - Some(vec![0, 2, 4]), - 1024, - None, - )?; - assert_eq!(13, csv.schema.fields().len()); - assert_eq!(3, csv.projected_schema.fields().len()); - assert_eq!(13, csv.file_schema().fields().len()); - assert_eq!(3, csv.schema().fields().len()); - let mut stream = csv.execute(0).await?; - let batch = stream.next().await.unwrap()?; - assert_eq!(3, batch.num_columns()); - let batch_schema = batch.schema(); - assert_eq!(3, batch_schema.fields().len()); - assert_eq!("c1", batch_schema.field(0).name()); - assert_eq!("c3", batch_schema.field(1).name()); - assert_eq!("c5", batch_schema.field(2).name()); - Ok(()) - } - - #[tokio::test] - async fn csv_exec_without_projection() -> Result<()> { - let schema = aggr_test_schema(); - let testdata = crate::test_util::arrow_test_data(); - let filename = "aggregate_test_100.csv"; - let path = format!("{}/csv/{}", testdata, filename); - let csv = CsvExec::try_new( - &path, - CsvReadOptions::new().schema(&schema), - None, - 1024, - None, - )?; - assert_eq!(13, csv.schema.fields().len()); - assert_eq!(13, csv.projected_schema.fields().len()); - assert_eq!(13, csv.file_schema().fields().len()); - assert_eq!(13, csv.schema().fields().len()); - let mut it = csv.execute(0).await?; - let batch = it.next().await.unwrap()?; - assert_eq!(13, batch.num_columns()); - let batch_schema = batch.schema(); - assert_eq!(13, batch_schema.fields().len()); - assert_eq!("c1", batch_schema.field(0).name()); - assert_eq!("c2", batch_schema.field(1).name()); - assert_eq!("c3", batch_schema.field(2).name()); - Ok(()) - } - - #[tokio::test] - async fn csv_exec_with_reader() -> Result<()> { - let schema = aggr_test_schema(); - let testdata = crate::test_util::arrow_test_data(); - let filename = "aggregate_test_100.csv"; - let path = format!("{}/csv/{}", testdata, filename); - let buf = std::fs::read(path).unwrap(); - let rdr = std::io::Cursor::new(buf); - let csv = CsvExec::try_new_from_reader( - rdr, - CsvReadOptions::new().schema(&schema), - Some(vec![0, 2, 4]), - 1024, - None, - )?; - assert_eq!(13, csv.schema.fields().len()); - assert_eq!(3, csv.projected_schema.fields().len()); - assert_eq!(13, csv.file_schema().fields().len()); - assert_eq!(3, csv.schema().fields().len()); - let mut stream = csv.execute(0).await?; - let batch = stream.next().await.unwrap()?; - assert_eq!(3, batch.num_columns()); - let batch_schema = batch.schema(); - assert_eq!(3, batch_schema.fields().len()); - assert_eq!("c1", batch_schema.field(0).name()); - assert_eq!("c3", batch_schema.field(1).name()); - assert_eq!("c5", batch_schema.field(2).name()); - Ok(()) - } -} diff --git a/datafusion/src/physical_plan/file_format/avro.rs b/datafusion/src/physical_plan/file_format/avro.rs index 23d912803250..0a57f8b386e0 100644 --- a/datafusion/src/physical_plan/file_format/avro.rs +++ b/datafusion/src/physical_plan/file_format/avro.rs @@ -16,8 +16,8 @@ // under the License. //! Execution plan for reading line-delimited Avro files -use crate::datasource::file_format::PartitionedFile; use crate::datasource::object_store::ObjectStore; +use crate::datasource::PartitionedFile; use crate::error::{DataFusionError, Result}; #[cfg(feature = "avro")] use crate::physical_plan::RecordBatchStream; @@ -82,6 +82,26 @@ impl AvroExec { limit, } } + /// List of data files + pub fn files(&self) -> &[PartitionedFile] { + &self.files + } + /// The schema before projection + pub fn file_schema(&self) -> &SchemaRef { + &self.schema + } + /// Optional projection for which columns to load + pub fn projection(&self) -> &Option> { + &self.projection + } + /// Batch size + pub fn batch_size(&self) -> usize { + self.batch_size + } + /// Limit in nr. of rows + pub fn limit(&self) -> Option { + self.limit + } } #[async_trait] @@ -156,14 +176,14 @@ impl ExecutionPlan for AvroExec { DisplayFormatType::Default => { write!( f, - "AvroExec: batch_size={}, limit={:?}, files=[{}]", - self.batch_size, - self.limit, + "AvroExec: files=[{}], batch_size={}, limit={:?}", self.files .iter() .map(|f| f.file_meta.path()) .collect::>() - .join(", ") + .join(", "), + self.batch_size, + self.limit, ) } } diff --git a/datafusion/src/physical_plan/file_format/csv.rs b/datafusion/src/physical_plan/file_format/csv.rs index df6022cc44aa..329dd6bae133 100644 --- a/datafusion/src/physical_plan/file_format/csv.rs +++ b/datafusion/src/physical_plan/file_format/csv.rs @@ -17,8 +17,8 @@ //! Execution plan for reading CSV files -use crate::datasource::file_format::PartitionedFile; use crate::datasource::object_store::ObjectStore; +use crate::datasource::PartitionedFile; use crate::error::{DataFusionError, Result}; use crate::physical_plan::{ DisplayFormatType, ExecutionPlan, Partitioning, RecordBatchStream, @@ -41,23 +41,16 @@ use async_trait::async_trait; #[derive(Debug, Clone)] pub struct CsvExec { object_store: Arc, - /// List of data files files: Vec, /// Schema representing the CSV file schema: SchemaRef, - /// Provided statistics + /// Schema after the projection has been applied + projected_schema: SchemaRef, statistics: Statistics, - /// Does the CSV file have a header? has_header: bool, - /// An optional column delimiter. Defaults to `b','` - delimiter: Option, - /// Optional projection for which columns to load + delimiter: u8, projection: Option>, - /// Schema after the projection has been applied - projected_schema: SchemaRef, - /// Batch size batch_size: usize, - /// Limit in nr. of rows limit: Option, } @@ -89,13 +82,42 @@ impl CsvExec { schema, statistics, has_header, - delimiter: Some(delimiter), + delimiter, projection, projected_schema, batch_size, limit, } } + + /// List of data files + pub fn files(&self) -> &[PartitionedFile] { + &self.files + } + /// The schema before projection + pub fn file_schema(&self) -> &SchemaRef { + &self.schema + } + /// true if the first line of each file is a header + pub fn has_header(&self) -> bool { + self.has_header + } + /// A column delimiter + pub fn delimiter(&self) -> u8 { + self.delimiter + } + /// Optional projection for which columns to load + pub fn projection(&self) -> &Option> { + &self.projection + } + /// Batch size + pub fn batch_size(&self) -> usize { + self.batch_size + } + /// Limit in nr. of rows + pub fn limit(&self) -> Option { + self.limit + } } #[async_trait] @@ -160,15 +182,15 @@ impl ExecutionPlan for CsvExec { DisplayFormatType::Default => { write!( f, - "CsvExec: has_header={}, batch_size={}, limit={:?}, files=[{}]", - self.has_header, - self.batch_size, - self.limit, + "CsvExec: files=[{}], has_header={}, batch_size={}, limit={:?}", self.files .iter() .map(|f| f.file_meta.path()) .collect::>() - .join(", ") + .join(", "), + self.has_header, + self.batch_size, + self.limit, ) } } @@ -191,7 +213,7 @@ impl CsvStream { reader: R, schema: SchemaRef, has_header: bool, - delimiter: Option, + delimiter: u8, projection: &Option>, batch_size: usize, limit: Option, @@ -203,7 +225,7 @@ impl CsvStream { reader, schema, has_header, - delimiter, + Some(delimiter), batch_size, bounds, projection.clone(), diff --git a/datafusion/src/physical_plan/file_format/json.rs b/datafusion/src/physical_plan/file_format/json.rs index 090fd3f59ef3..068e53a60e0c 100644 --- a/datafusion/src/physical_plan/file_format/json.rs +++ b/datafusion/src/physical_plan/file_format/json.rs @@ -19,8 +19,8 @@ use async_trait::async_trait; use futures::Stream; -use crate::datasource::file_format::PartitionedFile; use crate::datasource::object_store::ObjectStore; +use crate::datasource::PartitionedFile; use crate::error::{DataFusionError, Result}; use crate::physical_plan::{ DisplayFormatType, ExecutionPlan, Partitioning, RecordBatchStream, diff --git a/datafusion/src/physical_plan/file_format/parquet.rs b/datafusion/src/physical_plan/file_format/parquet.rs index 3d73d0550fa0..77eed01f893c 100644 --- a/datafusion/src/physical_plan/file_format/parquet.rs +++ b/datafusion/src/physical_plan/file_format/parquet.rs @@ -59,7 +59,7 @@ use tokio::{ use async_trait::async_trait; -use crate::datasource::file_format::{FilePartition, PartitionedFile}; +use crate::datasource::{FilePartition, PartitionedFile}; /// Execution plan for scanning one or more Parquet partitions #[derive(Debug, Clone)] @@ -202,6 +202,27 @@ impl ParquetExec { (Arc::new(projected_schema), statistics) } + + /// List of data files + pub fn partitions(&self) -> Vec<&[PartitionedFile]> { + self.partitions + .iter() + .map(|fp| fp.file_partition.files.as_slice()) + .collect() + } + /// Optional projection for which columns to load + pub fn projection(&self) -> &[usize] { + &self.projection + } + /// Batch size + pub fn batch_size(&self) -> usize { + self.batch_size + } + + /// Limit in nr. of rows + pub fn limit(&self) -> Option { + self.limit + } } impl ParquetPartition { diff --git a/datafusion/src/physical_plan/filter.rs b/datafusion/src/physical_plan/filter.rs index 8acfd1b92e6b..79b5ebc508f5 100644 --- a/datafusion/src/physical_plan/filter.rs +++ b/datafusion/src/physical_plan/filter.rs @@ -222,11 +222,12 @@ impl RecordBatchStream for FilterExecStream { mod tests { use super::*; - use crate::physical_plan::csv::{CsvExec, CsvReadOptions}; + use crate::datasource::object_store::local::LocalFileSystem; use crate::physical_plan::expressions::*; + use crate::physical_plan::file_format::CsvExec; use crate::physical_plan::ExecutionPlan; use crate::scalar::ScalarValue; - use crate::test; + use crate::test::{self, aggr_test_schema}; use crate::{logical_plan::Operator, physical_plan::collect}; use std::iter::Iterator; @@ -235,15 +236,20 @@ mod tests { let schema = test::aggr_test_schema(); let partitions = 4; - let path = test::create_partitioned_csv("aggregate_test_100.csv", partitions)?; - - let csv = CsvExec::try_new( - &path, - CsvReadOptions::new().schema(&schema), + let (_, files) = + test::create_partitioned_csv("aggregate_test_100.csv", partitions)?; + + let csv = CsvExec::new( + Arc::new(LocalFileSystem {}), + files, + Statistics::default(), + aggr_test_schema(), + true, + b',', None, 1024, None, - )?; + ); let predicate: Arc = binary( binary( diff --git a/datafusion/src/physical_plan/json.rs b/datafusion/src/physical_plan/json.rs deleted file mode 100644 index 675d88ec3bfa..000000000000 --- a/datafusion/src/physical_plan/json.rs +++ /dev/null @@ -1,507 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -//! Execution plan for reading line-delimited JSON files -use async_trait::async_trait; -use futures::Stream; - -use super::DisplayFormatType; -use super::{ - common, source::Source, ExecutionPlan, Partitioning, RecordBatchStream, Statistics, -}; -use crate::error::{DataFusionError, Result}; -use arrow::json::reader::{infer_json_schema_from_iterator, ValueIter}; -use arrow::{ - datatypes::{Schema, SchemaRef}, - error::Result as ArrowResult, - json, - record_batch::RecordBatch, -}; -use std::fs::File; -use std::{any::Any, io::Seek}; -use std::{ - io::{BufReader, Read}, - pin::Pin, - sync::{Arc, Mutex}, - task::{Context, Poll}, -}; - -/// Line-delimited JSON read options -#[derive(Clone)] -pub struct NdJsonReadOptions<'a> { - /// The data source schema. - pub schema: Option, - - /// Max number of rows to read from CSV files for schema inference if needed. Defaults to 1000. - pub schema_infer_max_records: usize, - - /// File extension; only files with this extension are selected for data input. - /// Defaults to ".json". - pub file_extension: &'a str, -} - -impl<'a> Default for NdJsonReadOptions<'a> { - fn default() -> Self { - Self { - schema: None, - schema_infer_max_records: 1000, - file_extension: ".json", - } - } -} - -trait SeekRead: Read + Seek {} - -impl SeekRead for T {} -/// Execution plan for scanning NdJson data source -#[derive(Debug)] -pub struct NdJsonExec { - source: Source>, - schema: SchemaRef, - projection: Option>, - projected_schema: SchemaRef, - file_extension: String, - batch_size: usize, - limit: Option, -} - -impl NdJsonExec { - /// Create a new execution plan for reading from a path - pub fn try_new( - path: &str, - options: NdJsonReadOptions, - projection: Option>, - batch_size: usize, - limit: Option, - ) -> Result { - let file_extension = options.file_extension.to_string(); - - let filenames = common::build_file_list(path, &file_extension)?; - - if filenames.is_empty() { - return Err(DataFusionError::Execution(format!( - "No files found at {path} with file extension {file_extension}", - path = path, - file_extension = file_extension.as_str() - ))); - } - - let schema = match options.schema { - Some(s) => s, - None => Arc::new(NdJsonExec::try_infer_schema( - filenames.clone(), - Some(options.schema_infer_max_records), - )?), - }; - - let projected_schema = match &projection { - None => schema.clone(), - Some(p) => Arc::new(Schema::new( - p.iter().map(|i| schema.field(*i).clone()).collect(), - )), - }; - - Ok(Self { - source: Source::PartitionedFiles { - path: path.to_string(), - filenames, - }, - schema, - file_extension, - projection, - projected_schema, - batch_size, - limit, - }) - } - /// Create a new execution plan for reading from a reader - pub fn try_new_from_reader( - reader: impl Read + Seek + Send + Sync + 'static, - options: NdJsonReadOptions, - projection: Option>, - batch_size: usize, - limit: Option, - ) -> Result { - let schema = match options.schema { - Some(s) => s, - None => { - return Err(DataFusionError::Execution( - "The schema must be provided in options when reading from a reader" - .to_string(), - )); - } - }; - - let projected_schema = match &projection { - None => schema.clone(), - Some(p) => Arc::new(Schema::new( - p.iter().map(|i| schema.field(*i).clone()).collect(), - )), - }; - - Ok(Self { - source: Source::Reader(Mutex::new(Some(Box::new(reader)))), - schema, - file_extension: String::new(), - projection, - projected_schema, - batch_size, - limit, - }) - } - - /// Path to directory containing partitioned CSV files with the same schema - pub fn path(&self) -> &str { - self.source.path() - } - - /// The individual files under path - pub fn filenames(&self) -> &[String] { - self.source.filenames() - } - - /// File extension - pub fn file_extension(&self) -> &str { - &self.file_extension - } - - /// Get the schema of the CSV file - pub fn file_schema(&self) -> SchemaRef { - self.schema.clone() - } - - /// Optional projection for which columns to load - pub fn projection(&self) -> Option<&Vec> { - self.projection.as_ref() - } - - /// Batch size - pub fn batch_size(&self) -> usize { - self.batch_size - } - - /// Limit - pub fn limit(&self) -> Option { - self.limit - } - - /// Infer schema for given CSV dataset - pub fn try_infer_schema( - mut filenames: Vec, - max_records: Option, - ) -> Result { - let mut schemas = Vec::new(); - let mut records_to_read = max_records.unwrap_or(usize::MAX); - while records_to_read > 0 && !filenames.is_empty() { - let file = File::open(filenames.pop().unwrap())?; - let mut reader = BufReader::new(file); - let iter = ValueIter::new(&mut reader, None); - let schema = infer_json_schema_from_iterator(iter.take_while(|_| { - let should_take = records_to_read > 0; - records_to_read -= 1; - should_take - }))?; - schemas.push(schema); - } - - Ok(Schema::try_merge(schemas)?) - } -} - -#[async_trait] -impl ExecutionPlan for NdJsonExec { - fn as_any(&self) -> &dyn Any { - self - } - - fn schema(&self) -> SchemaRef { - self.projected_schema.clone() - } - - fn output_partitioning(&self) -> Partitioning { - Partitioning::UnknownPartitioning(match &self.source { - Source::PartitionedFiles { filenames, .. } => filenames.len(), - Source::Reader(_) => 1, - }) - } - - fn children(&self) -> Vec> { - Vec::new() - } - - fn with_new_children( - &self, - children: Vec>, - ) -> Result> { - if !children.is_empty() { - Err(DataFusionError::Internal(format!( - "Children cannot be replaced in {:?}", - self - ))) - } else if let Source::PartitionedFiles { filenames, path } = &self.source { - Ok(Arc::new(Self { - source: Source::PartitionedFiles { - filenames: filenames.clone(), - path: path.clone(), - }, - schema: self.schema.clone(), - projection: self.projection.clone(), - projected_schema: self.projected_schema.clone(), - batch_size: self.batch_size, - limit: self.limit, - file_extension: self.file_extension.clone(), - })) - } else { - Err(DataFusionError::Internal( - "NdJsonExec with reader source cannot be used with `with_new_children`" - .to_string(), - )) - } - } - - 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(), - ); - } - match &self.source { - Source::PartitionedFiles { filenames, .. } => { - let file = File::open(&filenames[partition])?; - - Ok(Box::pin(NdJsonStream::new( - builder.build(file)?, - self.limit, - ))) - } - Source::Reader(rdr) => { - if partition != 0 { - Err(DataFusionError::Internal( - "Only partition 0 is valid when CSV comes from a reader" - .to_string(), - )) - } else if let Some(rdr) = rdr.lock().unwrap().take() { - Ok(Box::pin(NdJsonStream::new(builder.build(rdr)?, self.limit))) - } else { - Err(DataFusionError::Execution( - "Error reading CSV: Data can only be read a single time when the source is a reader" - .to_string(), - )) - } - } - } - } - - fn fmt_as( - &self, - t: DisplayFormatType, - f: &mut std::fmt::Formatter, - ) -> std::fmt::Result { - match t { - DisplayFormatType::Default => { - write!(f, "NdJsonExec: source={:?}", self.source) - } - } - } - - fn statistics(&self) -> Statistics { - // TODO stats: handle statistics - Statistics::default() - } -} - -struct NdJsonStream { - reader: json::Reader, - remain: Option, -} - -impl NdJsonStream { - fn new(reader: json::Reader, limit: Option) -> Self { - Self { - reader, - remain: limit, - } - } -} - -impl Stream for NdJsonStream { - type Item = ArrowResult; - - fn poll_next( - mut self: Pin<&mut Self>, - _cx: &mut Context<'_>, - ) -> Poll> { - if let Some(remain) = self.remain.as_mut() { - if *remain < 1 { - return Poll::Ready(None); - } - } - - Poll::Ready(match self.reader.next() { - Ok(Some(item)) => { - if let Some(remain) = self.remain.as_mut() { - if *remain >= item.num_rows() { - *remain -= item.num_rows(); - Some(Ok(item)) - } else { - let len = *remain; - *remain = 0; - Some(Ok(RecordBatch::try_new( - item.schema(), - item.columns() - .iter() - .map(|column| column.slice(0, len)) - .collect(), - )?)) - } - } else { - Some(Ok(item)) - } - } - Ok(None) => None, - Err(err) => Some(Err(err)), - }) - } -} - -impl RecordBatchStream for NdJsonStream { - fn schema(&self) -> SchemaRef { - self.reader.schema() - } -} - -#[cfg(test)] -mod tests { - use super::*; - use futures::StreamExt; - - const TEST_DATA_BASE: &str = "tests/jsons"; - - #[tokio::test] - async fn nd_json_exec_file_without_projection() -> Result<()> { - use arrow::datatypes::DataType; - let path = format!("{}/1.json", TEST_DATA_BASE); - let exec = NdJsonExec::try_new(&path, Default::default(), None, 1024, Some(3))?; - let inferred_schema = exec.schema(); - assert_eq!(inferred_schema.fields().len(), 4); - - // a,b,c,d should be inferred - inferred_schema.field_with_name("a").unwrap(); - inferred_schema.field_with_name("b").unwrap(); - inferred_schema.field_with_name("c").unwrap(); - inferred_schema.field_with_name("d").unwrap(); - - assert_eq!( - inferred_schema.field_with_name("a").unwrap().data_type(), - &DataType::Int64 - ); - assert!(matches!( - inferred_schema.field_with_name("b").unwrap().data_type(), - DataType::List(_) - )); - assert_eq!( - inferred_schema.field_with_name("d").unwrap().data_type(), - &DataType::Utf8 - ); - - let mut it = exec.execute(0).await?; - let batch = it.next().await.unwrap()?; - - assert_eq!(batch.num_rows(), 3); - let values = batch - .column(0) - .as_any() - .downcast_ref::() - .unwrap(); - assert_eq!(values.value(0), 1); - assert_eq!(values.value(1), -10); - assert_eq!(values.value(2), 2); - - Ok(()) - } - - #[tokio::test] - async fn nd_json_exec_file_projection() -> Result<()> { - let path = format!("{}/1.json", TEST_DATA_BASE); - let exec = - NdJsonExec::try_new(&path, Default::default(), Some(vec![0, 2]), 1024, None)?; - let inferred_schema = exec.schema(); - assert_eq!(inferred_schema.fields().len(), 2); - - inferred_schema.field_with_name("a").unwrap(); - inferred_schema.field_with_name("b").unwrap_err(); - inferred_schema.field_with_name("c").unwrap(); - inferred_schema.field_with_name("d").unwrap_err(); - - let mut it = exec.execute(0).await?; - let batch = it.next().await.unwrap()?; - - assert_eq!(batch.num_rows(), 4); - let values = batch - .column(0) - .as_any() - .downcast_ref::() - .unwrap(); - assert_eq!(values.value(0), 1); - assert_eq!(values.value(1), -10); - assert_eq!(values.value(2), 2); - Ok(()) - } - - #[tokio::test] - async fn nd_json_exec_from_reader() -> Result<()> { - let content = r#"{"a":"aaa", "b":[2.0, 1.3, -6.1], "c":[false, true], "d":"4"} -{"a":"bbb", "b":[2.0, 1.3, -6.1], "c":[true, true], "d":"4"}"#; - let cur = std::io::Cursor::new(content); - let mut bufrdr = std::io::BufReader::new(cur); - let schema = - arrow::json::reader::infer_json_schema_from_seekable(&mut bufrdr, None)?; - let exec = NdJsonExec::try_new_from_reader( - bufrdr, - NdJsonReadOptions { - schema: Some(Arc::new(schema)), - ..Default::default() - }, - None, - 1024, - Some(1), - )?; - - let mut it = exec.execute(0).await?; - let batch = it.next().await.unwrap()?; - - assert_eq!(batch.num_rows(), 1); - - let values = batch - .column(0) - .as_any() - .downcast_ref::() - .unwrap(); - assert_eq!(values.value(0), "aaa"); - - Ok(()) - } -} diff --git a/datafusion/src/physical_plan/limit.rs b/datafusion/src/physical_plan/limit.rs index ccd719f32468..bd48e4d2e5d4 100644 --- a/datafusion/src/physical_plan/limit.rs +++ b/datafusion/src/physical_plan/limit.rs @@ -384,9 +384,10 @@ mod tests { use common::collect; use super::*; + use crate::datasource::object_store::local::LocalFileSystem; use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; use crate::physical_plan::common; - use crate::physical_plan::csv::{CsvExec, CsvReadOptions}; + use crate::physical_plan::file_format::CsvExec; use crate::test; #[tokio::test] @@ -394,16 +395,20 @@ mod tests { let schema = test::aggr_test_schema(); let num_partitions = 4; - let path = + let (_, files) = test::create_partitioned_csv("aggregate_test_100.csv", num_partitions)?; - let csv = CsvExec::try_new( - &path, - CsvReadOptions::new().schema(&schema), + let csv = CsvExec::new( + Arc::new(LocalFileSystem {}), + files, + Statistics::default(), + schema, + true, + b',', None, 1024, None, - )?; + ); // input should have 4 partitions assert_eq!(csv.output_partitioning().partition_count(), num_partitions); diff --git a/datafusion/src/physical_plan/mod.rs b/datafusion/src/physical_plan/mod.rs index 373b5e3fe959..3accaadce607 100644 --- a/datafusion/src/physical_plan/mod.rs +++ b/datafusion/src/physical_plan/mod.rs @@ -201,7 +201,7 @@ pub trait ExecutionPlan: Debug + Send + Sync { /// let mut ctx = ExecutionContext::with_config(config); /// /// // register the a table -/// ctx.register_csv("example", "tests/example.csv", CsvReadOptions::new()).unwrap(); +/// ctx.register_csv("example", "tests/example.csv", CsvReadOptions::new()).await.unwrap(); /// /// // create a plan to run a SQL query /// let plan = ctx @@ -218,7 +218,7 @@ pub trait ExecutionPlan: Debug + Send + Sync { /// \n CoalesceBatchesExec: target_batch_size=4096\ /// \n FilterExec: a@0 < 5\ /// \n RepartitionExec: partitioning=RoundRobinBatch(3)\ -/// \n CsvExec: source=Path(tests/example.csv: [tests/example.csv]), has_header=true", +/// \n CsvExec: files=[tests/example.csv], has_header=true, batch_size=8192, limit=None", /// plan_string.trim()); /// } /// ``` @@ -606,14 +606,12 @@ pub trait Accumulator: Send + Sync + Debug { pub mod aggregates; pub mod analyze; pub mod array_expressions; -pub mod avro; pub mod coalesce_batches; pub mod coalesce_partitions; pub mod common; pub mod cross_join; #[cfg(feature = "crypto_expressions")] pub mod crypto_expressions; -pub mod csv; pub mod datetime_expressions; pub mod display; pub mod distinct_expressions; @@ -628,12 +626,10 @@ pub mod hash_join; pub mod hash_utils; pub(crate) mod hyperloglog; pub mod join_utils; -pub mod json; pub mod limit; pub mod math_expressions; pub mod memory; pub mod metrics; -pub mod parquet; pub mod planner; pub mod projection; #[cfg(feature = "regex_expressions")] @@ -641,7 +637,6 @@ pub mod regex_expressions; pub mod repartition; pub mod sort; pub mod sort_preserving_merge; -pub mod source; pub mod stream; pub mod string_expressions; pub mod type_coercion; diff --git a/datafusion/src/physical_plan/parquet.rs b/datafusion/src/physical_plan/parquet.rs deleted file mode 100644 index f4ac4c8fddaf..000000000000 --- a/datafusion/src/physical_plan/parquet.rs +++ /dev/null @@ -1,903 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -//! 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::{ - error::{DataFusionError, Result}, - logical_plan::{Column, Expr}, - physical_optimizer::pruning::{PruningPredicate, PruningStatistics}, - physical_plan::{ - DisplayFormatType, ExecutionPlan, Partitioning, SendableRecordBatchStream, - }, - scalar::ScalarValue, -}; - -use super::Statistics; - -use arrow::{ - array::ArrayRef, - datatypes::{Schema, SchemaRef}, - error::{ArrowError, Result as ArrowResult}, - record_batch::RecordBatch, -}; -use log::debug; -use parquet::file::{ - metadata::RowGroupMetaData, - reader::{FileReader, SerializedFileReader}, - statistics::Statistics as ParquetStatistics, -}; - -use fmt::Debug; -use parquet::arrow::{ArrowReader, ParquetFileArrowReader}; - -use tokio::{ - sync::mpsc::{channel, Receiver, Sender}, - task, -}; - -use async_trait::async_trait; - -use super::metrics::{self, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}; -use super::stream::RecordBatchReceiverStream; -use crate::datasource::parquet::ParquetTableDescriptor; -use crate::datasource::{get_statistics_with_limit, FilePartition, PartitionedFile}; - -/// Execution plan for scanning one or more Parquet partitions -#[derive(Debug, Clone)] -pub struct ParquetExec { - /// Parquet partitions to read - pub partitions: Vec, - /// Schema after projection is applied - pub schema: SchemaRef, - /// Projection for which columns to load - projection: Vec, - /// Batch size - batch_size: usize, - /// Statistics for the data set (sum of statistics for all partitions) - statistics: Statistics, - /// Execution metrics - metrics: ExecutionPlanMetricsSet, - /// Optional predicate builder - predicate_builder: Option, - /// Optional limit of the number of rows - limit: Option, -} - -/// Represents one partition of a Parquet data set and this currently means one Parquet file. -/// -/// In the future it would be good to support subsets of files based on ranges of row groups -/// so that we can better parallelize reads of large files across available cores (see -/// [ARROW-10995](https://issues.apache.org/jira/browse/ARROW-10995)). -/// -/// We may also want to support reading Parquet files that are partitioned based on a key and -/// in this case we would want this partition struct to represent multiple files for a given -/// partition key (see [ARROW-11019](https://issues.apache.org/jira/browse/ARROW-11019)). -#[derive(Debug, Clone)] -pub struct ParquetPartition { - /// The Parquet filename for this partition - pub file_partition: FilePartition, - /// Execution metrics - metrics: ExecutionPlanMetricsSet, -} - -/// Stores metrics about the parquet execution for a particular parquet file -#[derive(Debug, Clone)] -struct ParquetFileMetrics { - /// Number of times the predicate could not be evaluated - pub predicate_evaluation_errors: metrics::Count, - /// Number of row groups pruned using - pub row_groups_pruned: metrics::Count, -} - -impl ParquetExec { - /// Create a new Parquet reader execution plan based on the specified Parquet filename or - /// directory containing Parquet files - pub fn try_from_path( - path: &str, - projection: Option>, - predicate: Option, - batch_size: usize, - target_partitions: usize, - limit: Option, - ) -> Result { - // build a list of filenames from the specified path, which could be a single file or - // a directory containing one or more parquet files - let table_desc = ParquetTableDescriptor::new(path)?; - Self::try_new( - Arc::new(table_desc), - projection, - predicate, - batch_size, - target_partitions, - limit, - ) - } - - /// Create a new Parquet reader execution plan with root descriptor, provided partitions and schema - pub fn try_new( - desc: Arc, - projection: Option>, - predicate: Option, - batch_size: usize, - target_partitions: usize, - limit: Option, - ) -> Result { - debug!("Creating ParquetExec, desc: {:?}, projection {:?}, predicate: {:?}, limit: {:?}", - desc, projection, predicate, limit); - - let metrics = ExecutionPlanMetricsSet::new(); - let (all_files, statistics) = get_statistics_with_limit(&desc.descriptor, limit); - let schema = desc.schema(); - - let mut partitions = Vec::with_capacity(target_partitions); - let chunked_files = split_files(&all_files, target_partitions); - for (index, group) in chunked_files.iter().enumerate() { - partitions.push(ParquetPartition::new( - Vec::from(*group), - index, - metrics.clone(), - )); - } - - let metrics = ExecutionPlanMetricsSet::new(); - let predicate_creation_errors = - MetricBuilder::new(&metrics).global_counter("num_predicate_creation_errors"); - - let predicate_builder = predicate.and_then(|predicate_expr| { - match PruningPredicate::try_new(&predicate_expr, schema.clone()) { - Ok(predicate_builder) => Some(predicate_builder), - Err(e) => { - debug!( - "Could not create pruning predicate for {:?}: {}", - predicate_expr, e - ); - predicate_creation_errors.add(1); - None - } - } - }); - - Ok(Self::new( - partitions, - schema, - projection, - statistics, - metrics, - predicate_builder, - batch_size, - limit, - )) - } - - /// Create a new Parquet reader execution plan with provided partitions and schema - #[allow(clippy::too_many_arguments)] - pub fn new( - partitions: Vec, - schema: SchemaRef, - projection: Option>, - statistics: Statistics, - metrics: ExecutionPlanMetricsSet, - predicate_builder: Option, - batch_size: usize, - limit: Option, - ) -> Self { - let projection = match projection { - Some(p) => p, - None => (0..schema.fields().len()).collect(), - }; - - let projected_schema = Schema::new( - projection - .iter() - .map(|i| schema.field(*i).clone()) - .collect(), - ); - - let new_column_statistics = statistics.column_statistics.map(|stats| { - let mut projected_stats = Vec::with_capacity(projection.len()); - for proj in &projection { - projected_stats.push(stats[*proj].clone()); - } - projected_stats - }); - - let statistics = Statistics { - num_rows: statistics.num_rows, - total_byte_size: statistics.total_byte_size, - column_statistics: new_column_statistics, - is_exact: statistics.is_exact, - }; - - Self { - partitions, - schema: Arc::new(projected_schema), - projection, - metrics, - predicate_builder, - batch_size, - statistics, - limit, - } - } - - /// Parquet partitions to read - pub fn partitions(&self) -> &[ParquetPartition] { - &self.partitions - } - - /// Projection for which columns to load - pub fn projection(&self) -> &[usize] { - &self.projection - } - - /// Batch size - pub fn batch_size(&self) -> usize { - self.batch_size - } -} - -impl ParquetPartition { - /// Create a new parquet partition - pub fn new( - files: Vec, - index: usize, - metrics: ExecutionPlanMetricsSet, - ) -> Self { - Self { - file_partition: FilePartition { index, files }, - metrics, - } - } -} - -impl ParquetFileMetrics { - /// Create new metrics - pub fn new( - partition: usize, - filename: &str, - metrics: &ExecutionPlanMetricsSet, - ) -> Self { - let predicate_evaluation_errors = MetricBuilder::new(metrics) - .with_new_label("filename", filename.to_string()) - .counter("predicate_evaluation_errors", partition); - - let row_groups_pruned = MetricBuilder::new(metrics) - .with_new_label("filename", filename.to_string()) - .counter("row_groups_pruned", partition); - - Self { - predicate_evaluation_errors, - row_groups_pruned, - } - } -} - -#[async_trait] -impl ExecutionPlan for ParquetExec { - /// Return a reference to Any that can be used for downcasting - fn as_any(&self) -> &dyn Any { - self - } - - fn schema(&self) -> SchemaRef { - self.schema.clone() - } - - fn children(&self) -> Vec> { - // this is a leaf node and has no children - vec![] - } - - /// Get the output partitioning of this plan - fn output_partitioning(&self) -> Partitioning { - Partitioning::UnknownPartitioning(self.partitions.len()) - } - - fn with_new_children( - &self, - children: Vec>, - ) -> Result> { - if children.is_empty() { - Ok(Arc::new(self.clone())) - } else { - Err(DataFusionError::Internal(format!( - "Children cannot be replaced in {:?}", - self - ))) - } - } - - async fn execute(&self, partition_index: usize) -> Result { - // because the parquet implementation is not thread-safe, it is necessary to execute - // on a thread and communicate with channels - let (response_tx, response_rx): ( - Sender>, - Receiver>, - ) = channel(2); - - let partition = self.partitions[partition_index].clone(); - let metrics = self.metrics.clone(); - let projection = self.projection.clone(); - let predicate_builder = self.predicate_builder.clone(); - let batch_size = self.batch_size; - let limit = self.limit; - - task::spawn_blocking(move || { - if let Err(e) = read_partition( - partition_index, - partition, - metrics, - &projection, - &predicate_builder, - batch_size, - response_tx, - limit, - ) { - println!("Parquet reader thread terminated due to error: {:?}", e); - } - }); - - Ok(RecordBatchReceiverStream::create(&self.schema, response_rx)) - } - - fn fmt_as( - &self, - t: DisplayFormatType, - f: &mut std::fmt::Formatter, - ) -> std::fmt::Result { - match t { - DisplayFormatType::Default => { - let files: Vec<_> = self - .partitions - .iter() - .map(|pp| format!("{}", pp.file_partition)) - .collect(); - - write!( - f, - "ParquetExec: batch_size={}, limit={:?}, partitions=[{}]", - self.batch_size, - self.limit, - files.join(", ") - ) - } - } - } - - fn metrics(&self) -> Option { - Some(self.metrics.clone_inner()) - } - - fn statistics(&self) -> Statistics { - self.statistics.clone() - } -} - -fn send_result( - response_tx: &Sender>, - result: ArrowResult, -) -> Result<()> { - // Note this function is running on its own blockng tokio thread so blocking here is ok. - response_tx - .blocking_send(result) - .map_err(|e| DataFusionError::Execution(e.to_string()))?; - Ok(()) -} - -/// Wraps parquet statistics in a way -/// that implements [`PruningStatistics`] -struct RowGroupPruningStatistics<'a> { - row_group_metadata: &'a [RowGroupMetaData], - parquet_schema: &'a Schema, -} - -/// Extract the min/max statistics from a `ParquetStatistics` object -macro_rules! get_statistic { - ($column_statistics:expr, $func:ident, $bytes_func:ident) => {{ - if !$column_statistics.has_min_max_set() { - return None; - } - match $column_statistics { - ParquetStatistics::Boolean(s) => Some(ScalarValue::Boolean(Some(*s.$func()))), - ParquetStatistics::Int32(s) => Some(ScalarValue::Int32(Some(*s.$func()))), - ParquetStatistics::Int64(s) => Some(ScalarValue::Int64(Some(*s.$func()))), - // 96 bit ints not supported - ParquetStatistics::Int96(_) => None, - ParquetStatistics::Float(s) => Some(ScalarValue::Float32(Some(*s.$func()))), - ParquetStatistics::Double(s) => Some(ScalarValue::Float64(Some(*s.$func()))), - ParquetStatistics::ByteArray(s) => { - let s = std::str::from_utf8(s.$bytes_func()) - .map(|s| s.to_string()) - .ok(); - Some(ScalarValue::Utf8(s)) - } - // type not supported yet - ParquetStatistics::FixedLenByteArray(_) => None, - } - }}; -} - -// Extract the min or max value calling `func` or `bytes_func` on the ParquetStatistics as appropriate -macro_rules! get_min_max_values { - ($self:expr, $column:expr, $func:ident, $bytes_func:ident) => {{ - let (column_index, field) = if let Some((v, f)) = $self.parquet_schema.column_with_name(&$column.name) { - (v, f) - } else { - // Named column was not present - return None - }; - - let data_type = field.data_type(); - let null_scalar: ScalarValue = if let Ok(v) = data_type.try_into() { - v - } else { - // DataFusion doesn't have support for ScalarValues of the column type - return None - }; - - let scalar_values : Vec = $self.row_group_metadata - .iter() - .flat_map(|meta| { - meta.column(column_index).statistics() - }) - .map(|stats| { - get_statistic!(stats, $func, $bytes_func) - }) - .map(|maybe_scalar| { - // column either did't have statistics at all or didn't have min/max values - maybe_scalar.unwrap_or_else(|| null_scalar.clone()) - }) - .collect(); - - // ignore errors converting to arrays (e.g. different types) - ScalarValue::iter_to_array(scalar_values).ok() - }} -} - -impl<'a> PruningStatistics for RowGroupPruningStatistics<'a> { - fn min_values(&self, column: &Column) -> Option { - get_min_max_values!(self, column, min, min_bytes) - } - - fn max_values(&self, column: &Column) -> Option { - get_min_max_values!(self, column, max, max_bytes) - } - - fn num_containers(&self) -> usize { - self.row_group_metadata.len() - } -} - -fn build_row_group_predicate( - predicate_builder: &PruningPredicate, - metrics: ParquetFileMetrics, - row_group_metadata: &[RowGroupMetaData], -) -> Box bool> { - let parquet_schema = predicate_builder.schema().as_ref(); - - let pruning_stats = RowGroupPruningStatistics { - row_group_metadata, - parquet_schema, - }; - let predicate_values = predicate_builder.prune(&pruning_stats); - - match predicate_values { - Ok(values) => { - // NB: false means don't scan row group - let num_pruned = values.iter().filter(|&v| !*v).count(); - metrics.row_groups_pruned.add(num_pruned); - Box::new(move |_, i| values[i]) - } - // stats filter array could not be built - // return a closure which will not filter out any row groups - Err(e) => { - debug!("Error evaluating row group predicate values {}", e); - metrics.predicate_evaluation_errors.add(1); - Box::new(|_r, _i| true) - } - } -} - -#[allow(clippy::too_many_arguments)] -fn read_partition( - partition_index: usize, - partition: ParquetPartition, - metrics: ExecutionPlanMetricsSet, - projection: &[usize], - predicate_builder: &Option, - batch_size: usize, - response_tx: Sender>, - limit: Option, -) -> Result<()> { - 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)?; - if let Some(predicate_builder) = predicate_builder { - let row_group_predicate = build_row_group_predicate( - predicate_builder, - file_metrics, - file_reader.metadata().row_groups(), - ); - file_reader.filter_row_groups(&row_group_predicate); - } - let mut arrow_reader = ParquetFileArrowReader::new(Arc::new(file_reader)); - let mut batch_reader = arrow_reader - .get_record_reader_by_columns(projection.to_owned(), batch_size)?; - loop { - match batch_reader.next() { - Some(Ok(batch)) => { - total_rows += batch.num_rows(); - send_result(&response_tx, Ok(batch))?; - if limit.map(|l| total_rows >= l).unwrap_or(false) { - break 'outer; - } - } - None => { - break; - } - Some(Err(e)) => { - let err_msg = format!( - "Error reading batch from {}: {}", - partitioned_file, - e.to_string() - ); - // send error to operator - send_result( - &response_tx, - Err(ArrowError::ParquetError(err_msg.clone())), - )?; - // terminate thread with error - return Err(DataFusionError::Execution(err_msg)); - } - } - } - } - - // finished reading files (dropping response_tx will close - // channel) - Ok(()) -} - -fn split_files( - partitioned_files: &[PartitionedFile], - n: usize, -) -> Vec<&[PartitionedFile]> { - let mut chunk_size = partitioned_files.len() / n; - if partitioned_files.len() % n > 0 { - chunk_size += 1; - } - partitioned_files.chunks(chunk_size).collect() -} - -#[cfg(test)] -mod tests { - use super::*; - use arrow::datatypes::{DataType, Field}; - use futures::StreamExt; - use parquet::{ - basic::Type as PhysicalType, - file::{metadata::RowGroupMetaData, statistics::Statistics as ParquetStatistics}, - schema::types::SchemaDescPtr, - }; - - #[test] - fn test_split_files() { - let files = vec![ - PartitionedFile::from("a".to_string()), - PartitionedFile::from("b".to_string()), - PartitionedFile::from("c".to_string()), - PartitionedFile::from("d".to_string()), - PartitionedFile::from("e".to_string()), - ]; - - let chunks = split_files(&files, 1); - assert_eq!(1, chunks.len()); - assert_eq!(5, chunks[0].len()); - - let chunks = split_files(&files, 2); - assert_eq!(2, chunks.len()); - assert_eq!(3, chunks[0].len()); - assert_eq!(2, chunks[1].len()); - - let chunks = split_files(&files, 5); - assert_eq!(5, chunks.len()); - assert_eq!(1, chunks[0].len()); - assert_eq!(1, chunks[1].len()); - assert_eq!(1, chunks[2].len()); - assert_eq!(1, chunks[3].len()); - assert_eq!(1, chunks[4].len()); - - let chunks = split_files(&files, 123); - assert_eq!(5, chunks.len()); - assert_eq!(1, chunks[0].len()); - assert_eq!(1, chunks[1].len()); - assert_eq!(1, chunks[2].len()); - assert_eq!(1, chunks[3].len()); - assert_eq!(1, chunks[4].len()); - } - - #[tokio::test] - async fn test() -> Result<()> { - let testdata = crate::test_util::parquet_test_data(); - let filename = format!("{}/alltypes_plain.parquet", testdata); - let parquet_exec = ParquetExec::try_from_path( - &filename, - Some(vec![0, 1, 2]), - None, - 1024, - 4, - None, - )?; - assert_eq!(parquet_exec.output_partitioning().partition_count(), 1); - - let mut results = parquet_exec.execute(0).await?; - let batch = results.next().await.unwrap()?; - - assert_eq!(8, batch.num_rows()); - assert_eq!(3, batch.num_columns()); - - let schema = batch.schema(); - let field_names: Vec<&str> = - schema.fields().iter().map(|f| f.name().as_str()).collect(); - assert_eq!(vec!["id", "bool_col", "tinyint_col"], field_names); - - let batch = results.next().await; - assert!(batch.is_none()); - - let batch = results.next().await; - assert!(batch.is_none()); - - let batch = results.next().await; - assert!(batch.is_none()); - - Ok(()) - } - - fn parquet_file_metrics() -> ParquetFileMetrics { - let metrics = Arc::new(ExecutionPlanMetricsSet::new()); - ParquetFileMetrics::new(0, "file.parquet", &metrics) - } - - #[test] - fn row_group_predicate_builder_simple_expr() -> Result<()> { - use crate::logical_plan::{col, lit}; - // int > 1 => c1_max > 1 - let expr = col("c1").gt(lit(15)); - let schema = Schema::new(vec![Field::new("c1", DataType::Int32, false)]); - let predicate_builder = PruningPredicate::try_new(&expr, Arc::new(schema))?; - - let schema_descr = get_test_schema_descr(vec![("c1", PhysicalType::INT32)]); - let rgm1 = get_row_group_meta_data( - &schema_descr, - vec![ParquetStatistics::int32(Some(1), Some(10), None, 0, false)], - ); - let rgm2 = get_row_group_meta_data( - &schema_descr, - vec![ParquetStatistics::int32(Some(11), Some(20), None, 0, false)], - ); - let row_group_metadata = vec![rgm1, rgm2]; - let row_group_predicate = build_row_group_predicate( - &predicate_builder, - parquet_file_metrics(), - &row_group_metadata, - ); - let row_group_filter = row_group_metadata - .iter() - .enumerate() - .map(|(i, g)| row_group_predicate(g, i)) - .collect::>(); - assert_eq!(row_group_filter, vec![false, true]); - - Ok(()) - } - - #[test] - fn row_group_predicate_builder_missing_stats() -> Result<()> { - use crate::logical_plan::{col, lit}; - // int > 1 => c1_max > 1 - let expr = col("c1").gt(lit(15)); - let schema = Schema::new(vec![Field::new("c1", DataType::Int32, false)]); - let predicate_builder = PruningPredicate::try_new(&expr, Arc::new(schema))?; - - let schema_descr = get_test_schema_descr(vec![("c1", PhysicalType::INT32)]); - let rgm1 = get_row_group_meta_data( - &schema_descr, - vec![ParquetStatistics::int32(None, None, None, 0, false)], - ); - let rgm2 = get_row_group_meta_data( - &schema_descr, - vec![ParquetStatistics::int32(Some(11), Some(20), None, 0, false)], - ); - let row_group_metadata = vec![rgm1, rgm2]; - let row_group_predicate = build_row_group_predicate( - &predicate_builder, - parquet_file_metrics(), - &row_group_metadata, - ); - let row_group_filter = row_group_metadata - .iter() - .enumerate() - .map(|(i, g)| row_group_predicate(g, i)) - .collect::>(); - // missing statistics for first row group mean that the result from the predicate expression - // is null / undefined so the first row group can't be filtered out - assert_eq!(row_group_filter, vec![true, true]); - - Ok(()) - } - - #[test] - fn row_group_predicate_builder_partial_expr() -> Result<()> { - use crate::logical_plan::{col, lit}; - // test row group predicate with partially supported expression - // int > 1 and int % 2 => c1_max > 1 and true - let expr = col("c1").gt(lit(15)).and(col("c2").modulus(lit(2))); - let schema = Arc::new(Schema::new(vec![ - Field::new("c1", DataType::Int32, false), - Field::new("c2", DataType::Int32, false), - ])); - let predicate_builder = PruningPredicate::try_new(&expr, schema.clone())?; - - let schema_descr = get_test_schema_descr(vec![ - ("c1", PhysicalType::INT32), - ("c2", PhysicalType::INT32), - ]); - let rgm1 = get_row_group_meta_data( - &schema_descr, - vec![ - ParquetStatistics::int32(Some(1), Some(10), None, 0, false), - ParquetStatistics::int32(Some(1), Some(10), None, 0, false), - ], - ); - let rgm2 = get_row_group_meta_data( - &schema_descr, - vec![ - ParquetStatistics::int32(Some(11), Some(20), None, 0, false), - ParquetStatistics::int32(Some(11), Some(20), None, 0, false), - ], - ); - let row_group_metadata = vec![rgm1, rgm2]; - let row_group_predicate = build_row_group_predicate( - &predicate_builder, - parquet_file_metrics(), - &row_group_metadata, - ); - let row_group_filter = row_group_metadata - .iter() - .enumerate() - .map(|(i, g)| row_group_predicate(g, i)) - .collect::>(); - // the first row group is still filtered out because the predicate expression can be partially evaluated - // when conditions are joined using AND - assert_eq!(row_group_filter, vec![false, true]); - - // if conditions in predicate are joined with OR and an unsupported expression is used - // this bypasses the entire predicate expression and no row groups are filtered out - let expr = col("c1").gt(lit(15)).or(col("c2").modulus(lit(2))); - let predicate_builder = PruningPredicate::try_new(&expr, schema)?; - let row_group_predicate = build_row_group_predicate( - &predicate_builder, - parquet_file_metrics(), - &row_group_metadata, - ); - let row_group_filter = row_group_metadata - .iter() - .enumerate() - .map(|(i, g)| row_group_predicate(g, i)) - .collect::>(); - assert_eq!(row_group_filter, vec![true, true]); - - Ok(()) - } - - #[test] - fn row_group_predicate_builder_unsupported_type() -> Result<()> { - use crate::logical_plan::{col, lit}; - // test row group predicate with unsupported statistics type (boolean) - // where a null array is generated for some statistics columns - // int > 1 and bool = true => c1_max > 1 and null - let expr = col("c1").gt(lit(15)).and(col("c2").eq(lit(true))); - let schema = Arc::new(Schema::new(vec![ - Field::new("c1", DataType::Int32, false), - Field::new("c2", DataType::Boolean, false), - ])); - let predicate_builder = PruningPredicate::try_new(&expr, schema)?; - - let schema_descr = get_test_schema_descr(vec![ - ("c1", PhysicalType::INT32), - ("c2", PhysicalType::BOOLEAN), - ]); - let rgm1 = get_row_group_meta_data( - &schema_descr, - vec![ - ParquetStatistics::int32(Some(1), Some(10), None, 0, false), - ParquetStatistics::boolean(Some(false), Some(true), None, 0, false), - ], - ); - let rgm2 = get_row_group_meta_data( - &schema_descr, - vec![ - ParquetStatistics::int32(Some(11), Some(20), None, 0, false), - ParquetStatistics::boolean(Some(false), Some(true), None, 0, false), - ], - ); - let row_group_metadata = vec![rgm1, rgm2]; - let row_group_predicate = build_row_group_predicate( - &predicate_builder, - parquet_file_metrics(), - &row_group_metadata, - ); - let row_group_filter = row_group_metadata - .iter() - .enumerate() - .map(|(i, g)| row_group_predicate(g, i)) - .collect::>(); - // no row group is filtered out because the predicate expression can't be evaluated - // when a null array is generated for a statistics column, - // because the null values propagate to the end result, making the predicate result undefined - assert_eq!(row_group_filter, vec![true, true]); - - Ok(()) - } - - fn get_row_group_meta_data( - schema_descr: &SchemaDescPtr, - column_statistics: Vec, - ) -> RowGroupMetaData { - use parquet::file::metadata::ColumnChunkMetaData; - let mut columns = vec![]; - for (i, s) in column_statistics.iter().enumerate() { - let column = ColumnChunkMetaData::builder(schema_descr.column(i)) - .set_statistics(s.clone()) - .build() - .unwrap(); - columns.push(column); - } - RowGroupMetaData::builder(schema_descr.clone()) - .set_num_rows(1000) - .set_total_byte_size(2000) - .set_column_metadata(columns) - .build() - .unwrap() - } - - fn get_test_schema_descr(fields: Vec<(&str, PhysicalType)>) -> SchemaDescPtr { - use parquet::schema::types::{SchemaDescriptor, Type as SchemaType}; - let mut schema_fields = fields - .iter() - .map(|(n, t)| { - Arc::new(SchemaType::primitive_type_builder(n, *t).build().unwrap()) - }) - .collect::>(); - let schema = SchemaType::group_type_builder("schema") - .with_fields(&mut schema_fields) - .build() - .unwrap(); - - Arc::new(SchemaDescriptor::new(Arc::new(schema))) - } -} diff --git a/datafusion/src/physical_plan/planner.rs b/datafusion/src/physical_plan/planner.rs index 06f3a1ddd961..be8c588bfda5 100644 --- a/datafusion/src/physical_plan/planner.rs +++ b/datafusion/src/physical_plan/planner.rs @@ -1397,9 +1397,11 @@ fn tuple_err(value: (Result, Result)) -> Result<(T, R)> { #[cfg(test)] mod tests { use super::*; + use crate::datasource::object_store::local::LocalFileSystem; + use crate::execution::options::CsvReadOptions; use crate::logical_plan::{DFField, DFSchema, DFSchemaRef}; use crate::physical_plan::{ - csv::CsvReadOptions, expressions, DisplayFormatType, Partitioning, Statistics, + expressions, DisplayFormatType, Partitioning, Statistics, }; use crate::scalar::ScalarValue; use crate::{ @@ -1429,14 +1431,21 @@ mod tests { let path = format!("{}/csv/aggregate_test_100.csv", testdata); let options = CsvReadOptions::new().schema_infer_max_records(100); - let logical_plan = LogicalPlanBuilder::scan_csv(path, options, None)? - // filter clause needs the type coercion rule applied - .filter(col("c7").lt(lit(5_u8)))? - .project(vec![col("c1"), col("c2")])? - .aggregate(vec![col("c1")], vec![sum(col("c2"))])? - .sort(vec![col("c1").sort(true, true)])? - .limit(10)? - .build()?; + let logical_plan = LogicalPlanBuilder::scan_csv( + Arc::new(LocalFileSystem {}), + path, + options, + None, + 1, + ) + .await? + // filter clause needs the type coercion rule applied + .filter(col("c7").lt(lit(5_u8)))? + .project(vec![col("c1"), col("c2")])? + .aggregate(vec![col("c1")], vec![sum(col("c2"))])? + .sort(vec![col("c1").sort(true, true)])? + .limit(10)? + .build()?; let plan = plan(&logical_plan).await?; @@ -1474,9 +1483,16 @@ mod tests { let path = format!("{}/csv/aggregate_test_100.csv", testdata); let options = CsvReadOptions::new().schema_infer_max_records(100); - let logical_plan = LogicalPlanBuilder::scan_csv(path, options, None)? - .filter(col("c7").lt(col("c12")))? - .build()?; + let logical_plan = LogicalPlanBuilder::scan_csv( + Arc::new(LocalFileSystem {}), + path, + options, + None, + 1, + ) + .await? + .filter(col("c7").lt(col("c12")))? + .build()?; let plan = plan(&logical_plan).await?; @@ -1511,8 +1527,15 @@ mod tests { col("c1").like(col("c2")), ]; for case in cases { - let logical_plan = LogicalPlanBuilder::scan_csv(&path, options, None)? - .project(vec![case.clone()]); + let logical_plan = LogicalPlanBuilder::scan_csv( + Arc::new(LocalFileSystem {}), + &path, + options, + None, + 1, + ) + .await? + .project(vec![case.clone()]); let message = format!( "Expression {:?} expected to error due to impossible coercion", case @@ -1604,11 +1627,18 @@ mod tests { Expr::Literal(ScalarValue::Utf8(Some("a".to_string()))), Expr::Literal(ScalarValue::Int64(Some(1))), ]; - let logical_plan = LogicalPlanBuilder::scan_csv(&path, options, None)? - // filter clause needs the type coercion rule applied - .filter(col("c12").lt(lit(0.05)))? - .project(vec![col("c1").in_list(list, false)])? - .build()?; + let logical_plan = LogicalPlanBuilder::scan_csv( + Arc::new(LocalFileSystem {}), + &path, + options, + None, + 1, + ) + .await? + // filter clause needs the type coercion rule applied + .filter(col("c12").lt(lit(0.05)))? + .project(vec![col("c1").in_list(list, false)])? + .build()?; let execution_plan = plan(&logical_plan).await?; // verify that the plan correctly adds cast from Int64(1) to Utf8 let expected = "InListExpr { expr: Column { name: \"c1\", index: 0 }, list: [Literal { value: Utf8(\"a\") }, CastExpr { expr: Literal { value: Int64(1) }, cast_type: Utf8, cast_options: CastOptions { safe: false } }], negated: false }"; @@ -1619,11 +1649,18 @@ mod tests { Expr::Literal(ScalarValue::Boolean(Some(true))), Expr::Literal(ScalarValue::Utf8(Some("a".to_string()))), ]; - let logical_plan = LogicalPlanBuilder::scan_csv(path, options, None)? - // filter clause needs the type coercion rule applied - .filter(col("c12").lt(lit(0.05)))? - .project(vec![col("c12").lt_eq(lit(0.025)).in_list(list, false)])? - .build()?; + let logical_plan = LogicalPlanBuilder::scan_csv( + Arc::new(LocalFileSystem {}), + &path, + options, + None, + 1, + ) + .await? + // filter clause needs the type coercion rule applied + .filter(col("c12").lt(lit(0.05)))? + .project(vec![col("c12").lt_eq(lit(0.025)).in_list(list, false)])? + .build()?; let execution_plan = plan(&logical_plan).await; let expected_error = "Unsupported CAST from Utf8 to Boolean"; @@ -1647,11 +1684,14 @@ mod tests { let options = CsvReadOptions::new().schema_infer_max_records(100); let logical_plan = LogicalPlanBuilder::scan_csv_with_name( - path, + Arc::new(LocalFileSystem {}), + &path, options, None, "aggregate_test_100", - )? + 1, + ) + .await? .aggregate(vec![col("c1")], vec![sum(col("c2"))])? .build()?; @@ -1677,9 +1717,16 @@ mod tests { let path = format!("{}/csv/aggregate_test_100.csv", testdata); let options = CsvReadOptions::new().schema_infer_max_records(100); - let logical_plan = LogicalPlanBuilder::scan_csv(path, options, None)? - .aggregate(vec![col("c1")], vec![sum(col("c2"))])? - .build()?; + let logical_plan = LogicalPlanBuilder::scan_csv( + Arc::new(LocalFileSystem {}), + &path, + options, + None, + 1, + ) + .await? + .aggregate(vec![col("c1")], vec![sum(col("c2"))])? + .build()?; let execution_plan = plan(&logical_plan).await?; let formatted = format!("{:?}", execution_plan); diff --git a/datafusion/src/physical_plan/projection.rs b/datafusion/src/physical_plan/projection.rs index f24726123f9d..794d9a2ec68e 100644 --- a/datafusion/src/physical_plan/projection.rs +++ b/datafusion/src/physical_plan/projection.rs @@ -259,10 +259,11 @@ impl RecordBatchStream for ProjectionStream { mod tests { use super::*; - use crate::physical_plan::csv::{CsvExec, CsvReadOptions}; + use crate::datasource::object_store::local::LocalFileSystem; use crate::physical_plan::expressions::{self, col}; + use crate::physical_plan::file_format::CsvExec; use crate::scalar::ScalarValue; - use crate::test; + use crate::test::{self, aggr_test_schema}; use futures::future; #[tokio::test] @@ -270,15 +271,20 @@ mod tests { let schema = test::aggr_test_schema(); let partitions = 4; - let path = test::create_partitioned_csv("aggregate_test_100.csv", partitions)?; - - let csv = CsvExec::try_new( - &path, - CsvReadOptions::new().schema(&schema), + let (_, files) = + test::create_partitioned_csv("aggregate_test_100.csv", partitions)?; + + let csv = CsvExec::new( + Arc::new(LocalFileSystem {}), + files, + Statistics::default(), + aggr_test_schema(), + true, + b',', None, 1024, None, - )?; + ); // pick column c1 and name it column c1 in the output schema let projection = ProjectionExec::try_new( diff --git a/datafusion/src/physical_plan/sort.rs b/datafusion/src/physical_plan/sort.rs index 303255651194..68a42585b6c7 100644 --- a/datafusion/src/physical_plan/sort.rs +++ b/datafusion/src/physical_plan/sort.rs @@ -318,15 +318,12 @@ mod tests { use std::sync::Weak; use super::*; + use crate::datasource::object_store::local::LocalFileSystem; use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; use crate::physical_plan::expressions::col; use crate::physical_plan::memory::MemoryExec; - use crate::physical_plan::{ - collect, - csv::{CsvExec, CsvReadOptions}, - }; - use crate::test; - use crate::test::exec::BlockingExec; + use crate::physical_plan::{collect, file_format::CsvExec}; + use crate::test::{self, aggr_test_schema, exec::BlockingExec}; use arrow::array::*; use arrow::datatypes::*; use futures::FutureExt; @@ -335,14 +332,20 @@ mod tests { async fn test_sort() -> Result<()> { let schema = test::aggr_test_schema(); let partitions = 4; - let path = test::create_partitioned_csv("aggregate_test_100.csv", partitions)?; - let csv = CsvExec::try_new( - &path, - CsvReadOptions::new().schema(&schema), + let (_, files) = + test::create_partitioned_csv("aggregate_test_100.csv", partitions)?; + + let csv = CsvExec::new( + Arc::new(LocalFileSystem {}), + files, + Statistics::default(), + aggr_test_schema(), + true, + b',', None, 1024, None, - )?; + ); let sort_exec = Arc::new(SortExec::try_new( vec![ diff --git a/datafusion/src/physical_plan/sort_preserving_merge.rs b/datafusion/src/physical_plan/sort_preserving_merge.rs index f63695057a7d..f65faccc3be9 100644 --- a/datafusion/src/physical_plan/sort_preserving_merge.rs +++ b/datafusion/src/physical_plan/sort_preserving_merge.rs @@ -642,15 +642,15 @@ impl RecordBatchStream for SortPreservingMergeStream { #[cfg(test)] mod tests { + use crate::datasource::object_store::local::LocalFileSystem; use crate::physical_plan::metrics::MetricValue; use std::iter::FromIterator; use crate::arrow::array::{Int32Array, StringArray, TimestampNanosecondArray}; use crate::assert_batches_eq; - use crate::datasource::CsvReadOptions; use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; - use crate::physical_plan::csv::CsvExec; use crate::physical_plan::expressions::col; + use crate::physical_plan::file_format::CsvExec; use crate::physical_plan::memory::MemoryExec; use crate::physical_plan::sort::SortExec; use crate::physical_plan::{collect, common}; @@ -914,18 +914,20 @@ mod tests { async fn test_partition_sort() { let schema = test::aggr_test_schema(); let partitions = 4; - let path = + let (_, files) = test::create_partitioned_csv("aggregate_test_100.csv", partitions).unwrap(); - let csv = Arc::new( - CsvExec::try_new( - &path, - CsvReadOptions::new().schema(&schema), - None, - 1024, - None, - ) - .unwrap(), - ); + + let csv = Arc::new(CsvExec::new( + Arc::new(LocalFileSystem {}), + files, + Statistics::default(), + Arc::clone(&schema), + true, + b',', + None, + 1024, + None, + )); let sort = vec![ PhysicalSortExpr { @@ -984,18 +986,20 @@ mod tests { ) -> Arc { let schema = test::aggr_test_schema(); let partitions = 4; - let path = + let (_, files) = test::create_partitioned_csv("aggregate_test_100.csv", partitions).unwrap(); - let csv = Arc::new( - CsvExec::try_new( - &path, - CsvReadOptions::new().schema(&schema), - None, - 1024, - None, - ) - .unwrap(), - ); + + let csv = Arc::new(CsvExec::new( + Arc::new(LocalFileSystem {}), + files, + Statistics::default(), + schema, + true, + b',', + None, + 1024, + None, + )); let sorted = basic_sort(csv, sort).await; let split: Vec<_> = sizes.iter().map(|x| split_batch(&sorted, *x)).collect(); diff --git a/datafusion/src/physical_plan/source.rs b/datafusion/src/physical_plan/source.rs deleted file mode 100644 index 32fa9c37c8a2..000000000000 --- a/datafusion/src/physical_plan/source.rs +++ /dev/null @@ -1,90 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -//! Contains a `Source` enum represents where the data comes from. - -use std::{io::Read, sync::Mutex}; - -/// Source represents where the data comes from. -pub(crate) enum Source> { - /// The data comes from partitioned files - PartitionedFiles { - /// Path to directory containing partitioned files with the same schema - path: String, - /// The individual files under path - filenames: Vec, - }, - - /// The data comes from anything impl Read trait - Reader(Mutex>), -} - -impl std::fmt::Debug for Source { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - match self { - Source::PartitionedFiles { path, filenames } => f - .debug_struct("PartitionedFiles") - .field("path", path) - .field("filenames", filenames) - .finish()?, - Source::Reader(_) => f.write_str("Reader")?, - }; - Ok(()) - } -} -impl std::fmt::Display for Source { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - match self { - Source::PartitionedFiles { path, filenames } => { - write!(f, "Path({}: [{}])", path, filenames.join(",")) - } - Source::Reader(_) => { - write!(f, "Reader(...)") - } - } - } -} - -impl Clone for Source { - fn clone(&self) -> Self { - match self { - Source::PartitionedFiles { path, filenames } => Self::PartitionedFiles { - path: path.clone(), - filenames: filenames.clone(), - }, - Source::Reader(_) => Self::Reader(Mutex::new(None)), - } - } -} - -impl Source { - /// Path to directory containing partitioned files with the same schema - pub fn path(&self) -> &str { - match self { - Source::PartitionedFiles { path, .. } => path.as_str(), - Source::Reader(_) => "", - } - } - - /// The individual files under path - pub fn filenames(&self) -> &[String] { - match self { - Source::PartitionedFiles { filenames, .. } => filenames, - Source::Reader(_) => &[], - } - } -} diff --git a/datafusion/src/physical_plan/union.rs b/datafusion/src/physical_plan/union.rs index a2f5952b8090..43e23850b19e 100644 --- a/datafusion/src/physical_plan/union.rs +++ b/datafusion/src/physical_plan/union.rs @@ -218,12 +218,10 @@ fn stats_union(mut left: Statistics, right: Statistics) -> Statistics { #[cfg(test)] mod tests { use super::*; + use crate::datasource::object_store::{local::LocalFileSystem, ObjectStore}; use crate::test; use crate::{ - physical_plan::{ - collect, - csv::{CsvExec, CsvReadOptions}, - }, + physical_plan::{collect, file_format::CsvExec}, scalar::ScalarValue, }; use arrow::record_batch::RecordBatch; @@ -231,26 +229,35 @@ mod tests { #[tokio::test] async fn test_union_partitions() -> Result<()> { let schema = test::aggr_test_schema(); + let fs: Arc = Arc::new(LocalFileSystem {}); // Create csv's with different partitioning - let path = test::create_partitioned_csv("aggregate_test_100.csv", 4)?; - let path2 = test::create_partitioned_csv("aggregate_test_100.csv", 5)?; - - let csv = CsvExec::try_new( - &path, - CsvReadOptions::new().schema(&schema), + let (_, files) = test::create_partitioned_csv("aggregate_test_100.csv", 4)?; + let (_, files2) = test::create_partitioned_csv("aggregate_test_100.csv", 5)?; + + let csv = CsvExec::new( + Arc::clone(&fs), + files, + Statistics::default(), + Arc::clone(&schema), + true, + b',', None, 1024, None, - )?; - - let csv2 = CsvExec::try_new( - &path2, - CsvReadOptions::new().schema(&schema), + ); + + let csv2 = CsvExec::new( + Arc::clone(&fs), + files2, + Statistics::default(), + schema, + true, + b',', None, 1024, None, - )?; + ); let union_exec = Arc::new(UnionExec::new(vec![Arc::new(csv), Arc::new(csv2)])); diff --git a/datafusion/src/physical_plan/windows/mod.rs b/datafusion/src/physical_plan/windows/mod.rs index 0f6d9105fae2..3aa67cf28ba4 100644 --- a/datafusion/src/physical_plan/windows/mod.rs +++ b/datafusion/src/physical_plan/windows/mod.rs @@ -175,25 +175,31 @@ pub(crate) fn find_ranges_in_range<'a>( #[cfg(test)] mod tests { use super::*; + use crate::datasource::object_store::local::LocalFileSystem; use crate::physical_plan::aggregates::AggregateFunction; - use crate::physical_plan::collect; - use crate::physical_plan::csv::{CsvExec, CsvReadOptions}; use crate::physical_plan::expressions::col; - use crate::test; + use crate::physical_plan::file_format::CsvExec; + use crate::physical_plan::{collect, Statistics}; + use crate::test::{self, aggr_test_schema}; use arrow::array::*; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; fn create_test_schema(partitions: usize) -> Result<(Arc, SchemaRef)> { let schema = test::aggr_test_schema(); - let path = test::create_partitioned_csv("aggregate_test_100.csv", partitions)?; - let csv = CsvExec::try_new( - &path, - CsvReadOptions::new().schema(&schema), + let (_, files) = + test::create_partitioned_csv("aggregate_test_100.csv", partitions)?; + let csv = CsvExec::new( + Arc::new(LocalFileSystem {}), + files, + Statistics::default(), + aggr_test_schema(), + true, + b',', None, 1024, None, - )?; + ); let input = Arc::new(csv); Ok((input, schema)) diff --git a/datafusion/src/prelude.rs b/datafusion/src/prelude.rs index 02b9d4f3419e..8e47ed60ea2b 100644 --- a/datafusion/src/prelude.rs +++ b/datafusion/src/prelude.rs @@ -27,6 +27,8 @@ pub use crate::dataframe::DataFrame; pub use crate::execution::context::{ExecutionConfig, ExecutionContext}; +pub use crate::execution::options::AvroReadOptions; +pub use crate::execution::options::{CsvReadOptions, NdJsonReadOptions}; pub use crate::logical_plan::{ array, ascii, avg, bit_length, btrim, character_length, chr, col, concat, concat_ws, count, create_udf, date_part, date_trunc, digest, in_list, initcap, left, length, @@ -35,4 +37,3 @@ pub use crate::logical_plan::{ split_part, starts_with, strpos, substr, sum, to_hex, translate, trim, upper, Column, JoinType, Partitioning, }; -pub use crate::physical_plan::csv::CsvReadOptions; diff --git a/datafusion/src/test/mod.rs b/datafusion/src/test/mod.rs index e9a33745eeeb..917e7b1b3a50 100644 --- a/datafusion/src/test/mod.rs +++ b/datafusion/src/test/mod.rs @@ -17,7 +17,8 @@ //! Common unit test utility methods -use crate::datasource::{MemTable, TableProvider}; +use crate::datasource::object_store::local::local_file_meta; +use crate::datasource::{MemTable, PartitionedFile, TableProvider}; use crate::error::Result; use crate::logical_plan::{LogicalPlan, LogicalPlanBuilder}; use array::{ @@ -51,19 +52,24 @@ pub fn create_table_dual() -> Arc { } /// Generated partitioned copy of a CSV file -pub fn create_partitioned_csv(filename: &str, partitions: usize) -> Result { +pub fn create_partitioned_csv( + filename: &str, + partitions: usize, +) -> Result<(String, Vec)> { let testdata = crate::test_util::arrow_test_data(); let path = format!("{}/csv/{}", testdata, filename); let tmp_dir = TempDir::new()?; let mut writers = vec![]; + let mut files = vec![]; for i in 0..partitions { let filename = format!("partition-{}.csv", i); let filename = tmp_dir.path().join(&filename); let writer = BufWriter::new(File::create(&filename).unwrap()); writers.push(writer); + files.push(filename); } let f = File::open(&path)?; @@ -88,7 +94,15 @@ pub fn create_partitioned_csv(filename: &str, partitions: usize) -> Result TestOutput { println!("Planning sql {}", sql); - let logical_plan = self.ctx.sql(sql).expect("planning").to_logical_plan(); + let logical_plan = self.ctx.sql(sql).await.expect("planning").to_logical_plan(); self.run_test(logical_plan, sql).await } @@ -523,6 +523,7 @@ impl ContextWithParquet { let input = self .ctx .sql("SELECT * from t") + .await .expect("planning") .collect() .await diff --git a/datafusion/tests/provider_filter_pushdown.rs b/datafusion/tests/provider_filter_pushdown.rs index 653b96c39320..f1655c5267b3 100644 --- a/datafusion/tests/provider_filter_pushdown.rs +++ b/datafusion/tests/provider_filter_pushdown.rs @@ -173,7 +173,8 @@ async fn assert_provider_row_count(value: i64, expected_count: u64) -> Result<() ctx.register_table("data", Arc::new(provider))?; let sql_results = ctx - .sql(&format!("select count(*) from data where flag = {}", value))? + .sql(&format!("select count(*) from data where flag = {}", value)) + .await? .collect() .await?; diff --git a/datafusion/tests/sql.rs b/datafusion/tests/sql.rs index 6c85f35d6855..69f1ece3e551 100644 --- a/datafusion/tests/sql.rs +++ b/datafusion/tests/sql.rs @@ -35,17 +35,12 @@ use arrow::{ use datafusion::assert_batches_eq; use datafusion::assert_batches_sorted_eq; use datafusion::logical_plan::LogicalPlan; -#[cfg(feature = "avro")] -use datafusion::physical_plan::avro::AvroReadOptions; use datafusion::physical_plan::functions::Volatility; use datafusion::physical_plan::metrics::MetricValue; use datafusion::physical_plan::ExecutionPlan; use datafusion::physical_plan::ExecutionPlanVisitor; use datafusion::prelude::*; -use datafusion::{ - datasource::{csv::CsvReadOptions, MemTable}, - physical_plan::collect, -}; +use datafusion::{datasource::MemTable, physical_plan::collect}; use datafusion::{ error::{DataFusionError, Result}, physical_plan::ColumnarValue, @@ -124,7 +119,8 @@ async fn nyc() -> Result<()> { "tripdata", "file.csv", CsvReadOptions::new().schema(&schema), - )?; + ) + .await?; let logical_plan = ctx.create_logical_plan( "SELECT passenger_count, MIN(fare_amount), MAX(fare_amount) \ @@ -157,7 +153,7 @@ async fn nyc() -> Result<()> { #[tokio::test] async fn parquet_query() { let mut ctx = ExecutionContext::new(); - register_alltypes_parquet(&mut ctx); + register_alltypes_parquet(&mut ctx).await; // NOTE that string_col is actually a binary column and does not have the UTF8 logical type // so we need an explicit cast let sql = "SELECT id, CAST(string_col AS varchar) FROM alltypes_plain"; @@ -185,6 +181,7 @@ async fn parquet_single_nan_schema() { let mut ctx = ExecutionContext::new(); let testdata = datafusion::test_util::parquet_test_data(); ctx.register_parquet("single_nan", &format!("{}/single_nan.parquet", testdata)) + .await .unwrap(); let sql = "SELECT mycol FROM single_nan"; let plan = ctx.create_logical_plan(sql).unwrap(); @@ -206,6 +203,7 @@ async fn parquet_list_columns() { "list_columns", &format!("{}/list_columns.parquet", testdata), ) + .await .unwrap(); let schema = Arc::new(Schema::new(vec![ @@ -299,7 +297,7 @@ async fn parquet_list_columns() { #[tokio::test] async fn csv_select_nested() -> Result<()> { let mut ctx = ExecutionContext::new(); - register_aggregate_csv(&mut ctx)?; + register_aggregate_csv(&mut ctx).await?; let sql = "SELECT o1, o2, c3 FROM ( SELECT c1 AS o1, c2 + 1 AS o2, c3 @@ -331,7 +329,7 @@ async fn csv_select_nested() -> Result<()> { #[tokio::test] async fn csv_count_star() -> Result<()> { let mut ctx = ExecutionContext::new(); - register_aggregate_csv(&mut ctx)?; + register_aggregate_csv(&mut ctx).await?; let sql = "SELECT COUNT(*), COUNT(1) AS c, COUNT(c1) FROM aggregate_test_100"; let actual = execute_to_batches(&mut ctx, sql).await; let expected = vec![ @@ -348,7 +346,7 @@ async fn csv_count_star() -> Result<()> { #[tokio::test] async fn csv_query_with_predicate() -> Result<()> { let mut ctx = ExecutionContext::new(); - register_aggregate_csv(&mut ctx)?; + register_aggregate_csv(&mut ctx).await?; let sql = "SELECT c1, c12 FROM aggregate_test_100 WHERE c12 > 0.376 AND c12 < 0.4"; let actual = execute_to_batches(&mut ctx, sql).await; let expected = vec![ @@ -366,7 +364,7 @@ async fn csv_query_with_predicate() -> Result<()> { #[tokio::test] async fn csv_query_with_negative_predicate() -> Result<()> { let mut ctx = ExecutionContext::new(); - register_aggregate_csv(&mut ctx)?; + register_aggregate_csv(&mut ctx).await?; let sql = "SELECT c1, c4 FROM aggregate_test_100 WHERE c3 < -55 AND -c4 > 30000"; let actual = execute_to_batches(&mut ctx, sql).await; let expected = vec![ @@ -384,7 +382,7 @@ async fn csv_query_with_negative_predicate() -> Result<()> { #[tokio::test] async fn csv_query_with_negated_predicate() -> Result<()> { let mut ctx = ExecutionContext::new(); - register_aggregate_csv(&mut ctx)?; + register_aggregate_csv(&mut ctx).await?; let sql = "SELECT COUNT(1) FROM aggregate_test_100 WHERE NOT(c1 != 'a')"; let actual = execute_to_batches(&mut ctx, sql).await; let expected = vec![ @@ -401,7 +399,7 @@ async fn csv_query_with_negated_predicate() -> Result<()> { #[tokio::test] async fn csv_query_with_is_not_null_predicate() -> Result<()> { let mut ctx = ExecutionContext::new(); - register_aggregate_csv(&mut ctx)?; + register_aggregate_csv(&mut ctx).await?; let sql = "SELECT COUNT(1) FROM aggregate_test_100 WHERE c1 IS NOT NULL"; let actual = execute_to_batches(&mut ctx, sql).await; let expected = vec![ @@ -418,7 +416,7 @@ async fn csv_query_with_is_not_null_predicate() -> Result<()> { #[tokio::test] async fn csv_query_with_is_null_predicate() -> Result<()> { let mut ctx = ExecutionContext::new(); - register_aggregate_csv(&mut ctx)?; + register_aggregate_csv(&mut ctx).await?; let sql = "SELECT COUNT(1) FROM aggregate_test_100 WHERE c1 IS NULL"; let actual = execute_to_batches(&mut ctx, sql).await; let expected = vec![ @@ -435,7 +433,7 @@ async fn csv_query_with_is_null_predicate() -> Result<()> { #[tokio::test] async fn csv_query_group_by_int_min_max() -> Result<()> { let mut ctx = ExecutionContext::new(); - register_aggregate_csv(&mut ctx)?; + register_aggregate_csv(&mut ctx).await?; let sql = "SELECT c2, MIN(c12), MAX(c12) FROM aggregate_test_100 GROUP BY c2"; let actual = execute_to_batches(&mut ctx, sql).await; let expected = vec![ @@ -456,7 +454,7 @@ async fn csv_query_group_by_int_min_max() -> Result<()> { #[tokio::test] async fn csv_query_group_by_float32() -> Result<()> { let mut ctx = ExecutionContext::new(); - register_aggregate_simple_csv(&mut ctx)?; + register_aggregate_simple_csv(&mut ctx).await?; let sql = "SELECT COUNT(*) as cnt, c1 FROM aggregate_simple GROUP BY c1 ORDER BY cnt DESC"; @@ -481,7 +479,7 @@ async fn csv_query_group_by_float32() -> Result<()> { #[tokio::test] async fn select_all() -> Result<()> { let mut ctx = ExecutionContext::new(); - register_aggregate_simple_csv(&mut ctx)?; + register_aggregate_simple_csv(&mut ctx).await?; let sql = "SELECT c1 FROM aggregate_simple order by c1"; let actual_no_all = execute(&mut ctx, sql).await; @@ -497,7 +495,7 @@ async fn select_all() -> Result<()> { #[tokio::test] async fn select_distinct() -> Result<()> { let mut ctx = ExecutionContext::new(); - register_aggregate_simple_csv(&mut ctx)?; + register_aggregate_simple_csv(&mut ctx).await?; let sql = "SELECT DISTINCT * FROM aggregate_simple"; let mut actual = execute(&mut ctx, sql).await; @@ -514,7 +512,7 @@ async fn select_distinct() -> Result<()> { #[tokio::test] async fn select_distinct_simple_1() { let mut ctx = ExecutionContext::new(); - register_aggregate_simple_csv(&mut ctx).unwrap(); + register_aggregate_simple_csv(&mut ctx).await.unwrap(); let sql = "SELECT DISTINCT c1 FROM aggregate_simple order by c1"; let actual = execute_to_batches(&mut ctx, sql).await; @@ -536,7 +534,7 @@ async fn select_distinct_simple_1() { #[tokio::test] async fn select_distinct_simple_2() { let mut ctx = ExecutionContext::new(); - register_aggregate_simple_csv(&mut ctx).unwrap(); + register_aggregate_simple_csv(&mut ctx).await.unwrap(); let sql = "SELECT DISTINCT c1, c2 FROM aggregate_simple order by c1"; let actual = execute_to_batches(&mut ctx, sql).await; @@ -558,7 +556,7 @@ async fn select_distinct_simple_2() { #[tokio::test] async fn select_distinct_simple_3() { let mut ctx = ExecutionContext::new(); - register_aggregate_simple_csv(&mut ctx).unwrap(); + register_aggregate_simple_csv(&mut ctx).await.unwrap(); let sql = "SELECT distinct c3 FROM aggregate_simple order by c3"; let actual = execute_to_batches(&mut ctx, sql).await; @@ -577,7 +575,7 @@ async fn select_distinct_simple_3() { #[tokio::test] async fn select_distinct_simple_4() { let mut ctx = ExecutionContext::new(); - register_aggregate_simple_csv(&mut ctx).unwrap(); + register_aggregate_simple_csv(&mut ctx).await.unwrap(); let sql = "SELECT distinct c1+c2 as a FROM aggregate_simple"; let actual = execute_to_batches(&mut ctx, sql).await; @@ -612,7 +610,7 @@ async fn projection_same_fields() -> Result<()> { #[tokio::test] async fn csv_query_group_by_float64() -> Result<()> { let mut ctx = ExecutionContext::new(); - register_aggregate_simple_csv(&mut ctx)?; + register_aggregate_simple_csv(&mut ctx).await?; let sql = "SELECT COUNT(*) as cnt, c2 FROM aggregate_simple GROUP BY c2 ORDER BY cnt DESC"; @@ -637,7 +635,7 @@ async fn csv_query_group_by_float64() -> Result<()> { #[tokio::test] async fn csv_query_group_by_boolean() -> Result<()> { let mut ctx = ExecutionContext::new(); - register_aggregate_simple_csv(&mut ctx)?; + register_aggregate_simple_csv(&mut ctx).await?; let sql = "SELECT COUNT(*) as cnt, c3 FROM aggregate_simple GROUP BY c3 ORDER BY cnt DESC"; @@ -659,7 +657,7 @@ async fn csv_query_group_by_boolean() -> Result<()> { #[tokio::test] async fn csv_query_group_by_two_columns() -> Result<()> { let mut ctx = ExecutionContext::new(); - register_aggregate_csv(&mut ctx)?; + register_aggregate_csv(&mut ctx).await?; let sql = "SELECT c1, c2, MIN(c3) FROM aggregate_test_100 GROUP BY c1, c2"; let actual = execute_to_batches(&mut ctx, sql).await; let expected = vec![ @@ -700,7 +698,7 @@ async fn csv_query_group_by_two_columns() -> Result<()> { #[tokio::test] async fn csv_query_group_by_and_having() -> Result<()> { let mut ctx = ExecutionContext::new(); - register_aggregate_csv(&mut ctx)?; + register_aggregate_csv(&mut ctx).await?; let sql = "SELECT c1, MIN(c3) AS m FROM aggregate_test_100 GROUP BY c1 HAVING m < -100 AND MAX(c3) > 70"; let actual = execute_to_batches(&mut ctx, sql).await; let expected = vec![ @@ -718,7 +716,7 @@ async fn csv_query_group_by_and_having() -> Result<()> { #[tokio::test] async fn csv_query_group_by_and_having_and_where() -> Result<()> { let mut ctx = ExecutionContext::new(); - register_aggregate_csv(&mut ctx)?; + register_aggregate_csv(&mut ctx).await?; let sql = "SELECT c1, MIN(c3) AS m FROM aggregate_test_100 WHERE c1 IN ('a', 'b') @@ -739,7 +737,7 @@ async fn csv_query_group_by_and_having_and_where() -> Result<()> { #[tokio::test] async fn all_where_empty() -> Result<()> { let mut ctx = ExecutionContext::new(); - register_aggregate_csv(&mut ctx)?; + register_aggregate_csv(&mut ctx).await?; let sql = "SELECT * FROM aggregate_test_100 WHERE 1=2"; @@ -752,7 +750,7 @@ async fn all_where_empty() -> Result<()> { #[tokio::test] async fn csv_query_having_without_group_by() -> Result<()> { let mut ctx = ExecutionContext::new(); - register_aggregate_csv(&mut ctx)?; + register_aggregate_csv(&mut ctx).await?; let sql = "SELECT c1, c2, c3 FROM aggregate_test_100 HAVING c2 >= 4 AND c3 > 90"; let actual = execute_to_batches(&mut ctx, sql).await; let expected = vec![ @@ -773,7 +771,7 @@ async fn csv_query_having_without_group_by() -> Result<()> { #[tokio::test] async fn csv_query_avg_sqrt() -> Result<()> { let mut ctx = create_ctx()?; - register_aggregate_csv(&mut ctx)?; + register_aggregate_csv(&mut ctx).await?; let sql = "SELECT avg(custom_sqrt(c12)) FROM aggregate_test_100"; let mut actual = execute(&mut ctx, sql).await; actual.sort(); @@ -788,7 +786,7 @@ async fn csv_query_avg_sqrt() -> Result<()> { #[tokio::test] async fn csv_query_custom_udf_with_cast() -> Result<()> { let mut ctx = create_ctx()?; - register_aggregate_csv(&mut ctx)?; + register_aggregate_csv(&mut ctx).await?; let sql = "SELECT avg(custom_sqrt(c11)) FROM aggregate_test_100"; let actual = execute(&mut ctx, sql).await; let expected = vec![vec!["0.6584408483418833"]]; @@ -800,7 +798,7 @@ async fn csv_query_custom_udf_with_cast() -> Result<()> { #[tokio::test] async fn sqrt_f32_vs_f64() -> Result<()> { let mut ctx = create_ctx()?; - register_aggregate_csv(&mut ctx)?; + register_aggregate_csv(&mut ctx).await?; // sqrt(f32)'s plan passes let sql = "SELECT avg(sqrt(c11)) FROM aggregate_test_100"; let actual = execute(&mut ctx, sql).await; @@ -818,7 +816,7 @@ async fn sqrt_f32_vs_f64() -> Result<()> { async fn csv_query_error() -> Result<()> { // sin(utf8) should error let mut ctx = create_ctx()?; - register_aggregate_csv(&mut ctx)?; + register_aggregate_csv(&mut ctx).await?; let sql = "SELECT sin(c1) FROM aggregate_test_100"; let plan = ctx.create_logical_plan(sql); assert!(plan.is_err()); @@ -829,7 +827,7 @@ async fn csv_query_error() -> Result<()> { #[tokio::test] async fn csv_query_sqrt_sqrt() -> Result<()> { let mut ctx = create_ctx()?; - register_aggregate_csv(&mut ctx)?; + register_aggregate_csv(&mut ctx).await?; let sql = "SELECT sqrt(sqrt(c12)) FROM aggregate_test_100 LIMIT 1"; let actual = execute(&mut ctx, sql).await; // sqrt(sqrt(c12=0.9294097332465232)) = 0.9818650561397431 @@ -872,7 +870,7 @@ fn custom_sqrt(args: &[ColumnarValue]) -> Result { #[tokio::test] async fn csv_query_avg() -> Result<()> { let mut ctx = ExecutionContext::new(); - register_aggregate_csv(&mut ctx)?; + register_aggregate_csv(&mut ctx).await?; let sql = "SELECT avg(c12) FROM aggregate_test_100"; let mut actual = execute(&mut ctx, sql).await; actual.sort(); @@ -884,7 +882,7 @@ async fn csv_query_avg() -> Result<()> { #[tokio::test] async fn csv_query_group_by_avg() -> Result<()> { let mut ctx = ExecutionContext::new(); - register_aggregate_csv(&mut ctx)?; + register_aggregate_csv(&mut ctx).await?; let sql = "SELECT c1, avg(c12) FROM aggregate_test_100 GROUP BY c1"; let actual = execute_to_batches(&mut ctx, sql).await; let expected = vec![ @@ -905,7 +903,7 @@ async fn csv_query_group_by_avg() -> Result<()> { #[tokio::test] async fn csv_query_group_by_avg_with_projection() -> Result<()> { let mut ctx = ExecutionContext::new(); - register_aggregate_csv(&mut ctx)?; + register_aggregate_csv(&mut ctx).await?; let sql = "SELECT avg(c12), c1 FROM aggregate_test_100 GROUP BY c1"; let actual = execute_to_batches(&mut ctx, sql).await; let expected = vec![ @@ -926,7 +924,7 @@ async fn csv_query_group_by_avg_with_projection() -> Result<()> { #[tokio::test] async fn csv_query_avg_multi_batch() -> Result<()> { let mut ctx = ExecutionContext::new(); - register_aggregate_csv(&mut ctx)?; + register_aggregate_csv(&mut ctx).await?; let sql = "SELECT avg(c12) FROM aggregate_test_100"; let plan = ctx.create_logical_plan(sql).unwrap(); let plan = ctx.optimize(&plan).unwrap(); @@ -946,7 +944,7 @@ async fn csv_query_avg_multi_batch() -> Result<()> { #[tokio::test] async fn csv_query_nullif_divide_by_0() -> Result<()> { let mut ctx = ExecutionContext::new(); - register_aggregate_csv(&mut ctx)?; + register_aggregate_csv(&mut ctx).await?; let sql = "SELECT c8/nullif(c7, 0) FROM aggregate_test_100"; let actual = execute(&mut ctx, sql).await; let actual = &actual[80..90]; // We just want to compare rows 80-89 @@ -969,7 +967,7 @@ async fn csv_query_nullif_divide_by_0() -> Result<()> { #[tokio::test] async fn csv_query_count() -> Result<()> { let mut ctx = ExecutionContext::new(); - register_aggregate_csv(&mut ctx)?; + register_aggregate_csv(&mut ctx).await?; let sql = "SELECT count(c12) FROM aggregate_test_100"; let actual = execute_to_batches(&mut ctx, sql).await; let expected = vec![ @@ -1004,7 +1002,7 @@ async fn csv_query_approx_count() -> Result<()> { #[tokio::test] async fn csv_query_window_with_empty_over() -> Result<()> { let mut ctx = ExecutionContext::new(); - register_aggregate_csv(&mut ctx)?; + register_aggregate_csv(&mut ctx).await?; let sql = "select \ c9, \ count(c5) over (), \ @@ -1033,7 +1031,7 @@ async fn csv_query_window_with_empty_over() -> Result<()> { #[tokio::test] async fn csv_query_window_with_partition_by() -> Result<()> { let mut ctx = ExecutionContext::new(); - register_aggregate_csv(&mut ctx)?; + register_aggregate_csv(&mut ctx).await?; let sql = "select \ c9, \ sum(cast(c4 as Int)) over (partition by c3), \ @@ -1063,7 +1061,7 @@ async fn csv_query_window_with_partition_by() -> Result<()> { #[tokio::test] async fn csv_query_window_with_order_by() -> Result<()> { let mut ctx = ExecutionContext::new(); - register_aggregate_csv(&mut ctx)?; + register_aggregate_csv(&mut ctx).await?; let sql = "select \ c9, \ sum(c5) over (order by c9), \ @@ -1096,7 +1094,7 @@ async fn csv_query_window_with_order_by() -> Result<()> { #[tokio::test] async fn csv_query_window_with_partition_by_order_by() -> Result<()> { let mut ctx = ExecutionContext::new(); - register_aggregate_csv(&mut ctx)?; + register_aggregate_csv(&mut ctx).await?; let sql = "select \ c9, \ sum(c5) over (partition by c4 order by c9), \ @@ -1129,7 +1127,7 @@ async fn csv_query_window_with_partition_by_order_by() -> Result<()> { #[tokio::test] async fn csv_query_group_by_int_count() -> Result<()> { let mut ctx = ExecutionContext::new(); - register_aggregate_csv(&mut ctx)?; + register_aggregate_csv(&mut ctx).await?; let sql = "SELECT c1, count(c12) FROM aggregate_test_100 GROUP BY c1"; let actual = execute_to_batches(&mut ctx, sql).await; let expected = vec![ @@ -1150,7 +1148,7 @@ async fn csv_query_group_by_int_count() -> Result<()> { #[tokio::test] async fn csv_query_group_with_aliased_aggregate() -> Result<()> { let mut ctx = ExecutionContext::new(); - register_aggregate_csv(&mut ctx)?; + register_aggregate_csv(&mut ctx).await?; let sql = "SELECT c1, count(c12) AS count FROM aggregate_test_100 GROUP BY c1"; let actual = execute_to_batches(&mut ctx, sql).await; let expected = vec![ @@ -1171,7 +1169,7 @@ async fn csv_query_group_with_aliased_aggregate() -> Result<()> { #[tokio::test] async fn csv_query_group_by_string_min_max() -> Result<()> { let mut ctx = ExecutionContext::new(); - register_aggregate_csv(&mut ctx)?; + register_aggregate_csv(&mut ctx).await?; let sql = "SELECT c1, MIN(c12), MAX(c12) FROM aggregate_test_100 GROUP BY c1"; let actual = execute_to_batches(&mut ctx, sql).await; let expected = vec![ @@ -1194,7 +1192,7 @@ async fn csv_query_group_by_string_min_max() -> Result<()> { #[tokio::test] async fn csv_query_cast() -> Result<()> { let mut ctx = ExecutionContext::new(); - register_aggregate_csv(&mut ctx)?; + register_aggregate_csv(&mut ctx).await?; let sql = "SELECT CAST(c12 AS float) FROM aggregate_test_100 WHERE c12 > 0.376 AND c12 < 0.4"; let actual = execute(&mut ctx, sql).await; let expected = vec![vec!["0.39144436569161134"], vec!["0.38870280983958583"]]; @@ -1205,7 +1203,7 @@ async fn csv_query_cast() -> Result<()> { #[tokio::test] async fn csv_query_cast_literal() -> Result<()> { let mut ctx = ExecutionContext::new(); - register_aggregate_csv(&mut ctx)?; + register_aggregate_csv(&mut ctx).await?; let sql = "SELECT c12, CAST(1 AS float) FROM aggregate_test_100 WHERE c12 > CAST(0 AS float) LIMIT 2"; let actual = execute(&mut ctx, sql).await; @@ -1412,7 +1410,7 @@ async fn union_all() -> Result<()> { #[tokio::test] async fn csv_union_all() -> Result<()> { let mut ctx = ExecutionContext::new(); - register_aggregate_csv(&mut ctx)?; + register_aggregate_csv(&mut ctx).await?; let sql = "SELECT c1 FROM aggregate_test_100 UNION ALL SELECT c1 FROM aggregate_test_100"; let actual = execute(&mut ctx, sql).await; @@ -1423,7 +1421,7 @@ async fn csv_union_all() -> Result<()> { #[tokio::test] async fn csv_query_limit() -> Result<()> { let mut ctx = ExecutionContext::new(); - register_aggregate_csv(&mut ctx)?; + register_aggregate_csv(&mut ctx).await?; let sql = "SELECT c1 FROM aggregate_test_100 LIMIT 2"; let actual = execute(&mut ctx, sql).await; let expected = vec![vec!["c"], vec!["d"]]; @@ -1434,7 +1432,7 @@ async fn csv_query_limit() -> Result<()> { #[tokio::test] async fn csv_query_limit_bigger_than_nbr_of_rows() -> Result<()> { let mut ctx = ExecutionContext::new(); - register_aggregate_csv(&mut ctx)?; + register_aggregate_csv(&mut ctx).await?; let sql = "SELECT c2 FROM aggregate_test_100 LIMIT 200"; let actual = execute(&mut ctx, sql).await; let expected = vec![ @@ -1546,7 +1544,7 @@ async fn csv_query_limit_bigger_than_nbr_of_rows() -> Result<()> { #[tokio::test] async fn csv_query_limit_with_same_nbr_of_rows() -> Result<()> { let mut ctx = ExecutionContext::new(); - register_aggregate_csv(&mut ctx)?; + register_aggregate_csv(&mut ctx).await?; let sql = "SELECT c2 FROM aggregate_test_100 LIMIT 100"; let actual = execute(&mut ctx, sql).await; let expected = vec![ @@ -1658,7 +1656,7 @@ async fn csv_query_limit_with_same_nbr_of_rows() -> Result<()> { #[tokio::test] async fn csv_query_limit_zero() -> Result<()> { let mut ctx = ExecutionContext::new(); - register_aggregate_csv(&mut ctx)?; + register_aggregate_csv(&mut ctx).await?; let sql = "SELECT c1 FROM aggregate_test_100 LIMIT 0"; let actual = execute(&mut ctx, sql).await; let expected: Vec> = vec![]; @@ -2445,14 +2443,14 @@ async fn csv_explain() { "logical_plan", "Projection: #aggregate_test_100.c1\ \n Filter: #aggregate_test_100.c2 > Int64(10)\ - \n TableScan: aggregate_test_100 projection=Some([0, 1])" + \n TableScan: aggregate_test_100 projection=Some([0, 1]), filters=[#aggregate_test_100.c2 > Int64(10)]" ], vec!["physical_plan", "ProjectionExec: expr=[c1@0 as c1]\ \n CoalesceBatchesExec: target_batch_size=4096\ \n FilterExec: CAST(c2@1 AS Int64) > 10\ \n RepartitionExec: partitioning=RoundRobinBatch(NUM_CORES)\ - \n CsvExec: source=Path(ARROW_TEST_DATA/csv/aggregate_test_100.csv: [ARROW_TEST_DATA/csv/aggregate_test_100.csv]), has_header=true\ + \n CsvExec: files=[ARROW_TEST_DATA/csv/aggregate_test_100.csv], has_header=true, batch_size=8192, limit=None\ \n" ]]; assert_eq!(expected, actual); @@ -2756,7 +2754,7 @@ async fn csv_explain_plans() { "Explain [plan_type:Utf8, plan:Utf8]", " Projection: #aggregate_test_100.c1 [c1:Utf8]", " Filter: #aggregate_test_100.c2 > Int64(10) [c1:Utf8, c2:Int32]", - " TableScan: aggregate_test_100 projection=Some([0, 1]) [c1:Utf8, c2:Int32]", + " TableScan: aggregate_test_100 projection=Some([0, 1]), filters=[#aggregate_test_100.c2 > Int64(10)] [c1:Utf8, c2:Int32]", ]; let formatted = plan.display_indent_schema().to_string(); let actual: Vec<&str> = formatted.trim().lines().collect(); @@ -2771,7 +2769,7 @@ async fn csv_explain_plans() { "Explain", " Projection: #aggregate_test_100.c1", " Filter: #aggregate_test_100.c2 > Int64(10)", - " TableScan: aggregate_test_100 projection=Some([0, 1])", + " TableScan: aggregate_test_100 projection=Some([0, 1]), filters=[#aggregate_test_100.c2 > Int64(10)]", ]; let formatted = plan.display_indent().to_string(); let actual: Vec<&str> = formatted.trim().lines().collect(); @@ -2793,7 +2791,7 @@ async fn csv_explain_plans() { " 2 -> 3 [arrowhead=none, arrowtail=normal, dir=back]", " 4[shape=box label=\"Filter: #aggregate_test_100.c2 > Int64(10)\"]", " 3 -> 4 [arrowhead=none, arrowtail=normal, dir=back]", - " 5[shape=box label=\"TableScan: aggregate_test_100 projection=Some([0, 1])\"]", + " 5[shape=box label=\"TableScan: aggregate_test_100 projection=Some([0, 1]), filters=[#aggregate_test_100.c2 > Int64(10)]\"]", " 4 -> 5 [arrowhead=none, arrowtail=normal, dir=back]", " }", " subgraph cluster_6", @@ -2804,7 +2802,7 @@ async fn csv_explain_plans() { " 7 -> 8 [arrowhead=none, arrowtail=normal, dir=back]", " 9[shape=box label=\"Filter: #aggregate_test_100.c2 > Int64(10)\\nSchema: [c1:Utf8, c2:Int32]\"]", " 8 -> 9 [arrowhead=none, arrowtail=normal, dir=back]", - " 10[shape=box label=\"TableScan: aggregate_test_100 projection=Some([0, 1])\\nSchema: [c1:Utf8, c2:Int32]\"]", + " 10[shape=box label=\"TableScan: aggregate_test_100 projection=Some([0, 1]), filters=[#aggregate_test_100.c2 > Int64(10)]\\nSchema: [c1:Utf8, c2:Int32]\"]", " 9 -> 10 [arrowhead=none, arrowtail=normal, dir=back]", " }", "}", @@ -2953,7 +2951,7 @@ async fn csv_explain_verbose_plans() { "Explain [plan_type:Utf8, plan:Utf8]", " Projection: #aggregate_test_100.c1 [c1:Utf8]", " Filter: #aggregate_test_100.c2 > Int64(10) [c1:Utf8, c2:Int32]", - " TableScan: aggregate_test_100 projection=Some([0, 1]) [c1:Utf8, c2:Int32]", + " TableScan: aggregate_test_100 projection=Some([0, 1]), filters=[#aggregate_test_100.c2 > Int64(10)] [c1:Utf8, c2:Int32]", ]; let formatted = plan.display_indent_schema().to_string(); let actual: Vec<&str> = formatted.trim().lines().collect(); @@ -2968,7 +2966,7 @@ async fn csv_explain_verbose_plans() { "Explain", " Projection: #aggregate_test_100.c1", " Filter: #aggregate_test_100.c2 > Int64(10)", - " TableScan: aggregate_test_100 projection=Some([0, 1])", + " TableScan: aggregate_test_100 projection=Some([0, 1]), filters=[#aggregate_test_100.c2 > Int64(10)]", ]; let formatted = plan.display_indent().to_string(); let actual: Vec<&str> = formatted.trim().lines().collect(); @@ -2990,7 +2988,7 @@ async fn csv_explain_verbose_plans() { " 2 -> 3 [arrowhead=none, arrowtail=normal, dir=back]", " 4[shape=box label=\"Filter: #aggregate_test_100.c2 > Int64(10)\"]", " 3 -> 4 [arrowhead=none, arrowtail=normal, dir=back]", - " 5[shape=box label=\"TableScan: aggregate_test_100 projection=Some([0, 1])\"]", + " 5[shape=box label=\"TableScan: aggregate_test_100 projection=Some([0, 1]), filters=[#aggregate_test_100.c2 > Int64(10)]\"]", " 4 -> 5 [arrowhead=none, arrowtail=normal, dir=back]", " }", " subgraph cluster_6", @@ -3001,7 +2999,7 @@ async fn csv_explain_verbose_plans() { " 7 -> 8 [arrowhead=none, arrowtail=normal, dir=back]", " 9[shape=box label=\"Filter: #aggregate_test_100.c2 > Int64(10)\\nSchema: [c1:Utf8, c2:Int32]\"]", " 8 -> 9 [arrowhead=none, arrowtail=normal, dir=back]", - " 10[shape=box label=\"TableScan: aggregate_test_100 projection=Some([0, 1])\\nSchema: [c1:Utf8, c2:Int32]\"]", + " 10[shape=box label=\"TableScan: aggregate_test_100 projection=Some([0, 1]), filters=[#aggregate_test_100.c2 > Int64(10)]\\nSchema: [c1:Utf8, c2:Int32]\"]", " 9 -> 10 [arrowhead=none, arrowtail=normal, dir=back]", " }", "}", @@ -3040,7 +3038,7 @@ async fn explain_analyze_runs_optimizers() { // repro for https://github.com/apache/arrow-datafusion/issues/917 // where EXPLAIN ANALYZE was not correctly running optiimizer let mut ctx = ExecutionContext::new(); - register_alltypes_parquet(&mut ctx); + register_alltypes_parquet(&mut ctx).await; // This happens as an optimization pass where count(*) can be // answered using statistics only. @@ -3105,6 +3103,7 @@ async fn register_aggregate_csv_by_sql(ctx: &mut ExecutionContext) { ", testdata )) + .await .expect("Creating dataframe for CREATE EXTERNAL TABLE"); // Mimic the CLI and execute the resulting plan -- even though it @@ -3116,18 +3115,19 @@ async fn register_aggregate_csv_by_sql(ctx: &mut ExecutionContext) { ); } -fn register_aggregate_csv(ctx: &mut ExecutionContext) -> Result<()> { +async fn register_aggregate_csv(ctx: &mut ExecutionContext) -> Result<()> { let testdata = datafusion::test_util::arrow_test_data(); let schema = aggr_test_schema(); ctx.register_csv( "aggregate_test_100", &format!("{}/csv/aggregate_test_100.csv", testdata), CsvReadOptions::new().schema(&schema), - )?; + ) + .await?; Ok(()) } -fn register_aggregate_simple_csv(ctx: &mut ExecutionContext) -> Result<()> { +async fn register_aggregate_simple_csv(ctx: &mut ExecutionContext) -> Result<()> { // It's not possible to use aggregate_test_100, not enought similar values to test grouping on floats let schema = Arc::new(Schema::new(vec![ Field::new("c1", DataType::Float32, false), @@ -3139,27 +3139,30 @@ fn register_aggregate_simple_csv(ctx: &mut ExecutionContext) -> Result<()> { "aggregate_simple", "tests/aggregate_simple.csv", CsvReadOptions::new().schema(&schema), - )?; + ) + .await?; Ok(()) } -fn register_alltypes_parquet(ctx: &mut ExecutionContext) { +async fn register_alltypes_parquet(ctx: &mut ExecutionContext) { let testdata = datafusion::test_util::parquet_test_data(); ctx.register_parquet( "alltypes_plain", &format!("{}/alltypes_plain.parquet", testdata), ) + .await .unwrap(); } #[cfg(feature = "avro")] -fn register_alltypes_avro(ctx: &mut ExecutionContext) { +async fn register_alltypes_avro(ctx: &mut ExecutionContext) { let testdata = datafusion::test_util::arrow_test_data(); ctx.register_avro( "alltypes_plain", &format!("{}/avro/alltypes_plain.avro", testdata), AvroReadOptions::default(), ) + .await .unwrap(); } @@ -3892,7 +3895,7 @@ where #[tokio::test] async fn csv_between_expr() -> Result<()> { let mut ctx = ExecutionContext::new(); - register_aggregate_csv(&mut ctx)?; + register_aggregate_csv(&mut ctx).await?; let sql = "SELECT c4 FROM aggregate_test_100 WHERE c12 BETWEEN 0.995 AND 1.0"; let mut actual = execute(&mut ctx, sql).await; actual.sort(); @@ -3904,7 +3907,7 @@ async fn csv_between_expr() -> Result<()> { #[tokio::test] async fn csv_between_expr_negated() -> Result<()> { let mut ctx = ExecutionContext::new(); - register_aggregate_csv(&mut ctx)?; + register_aggregate_csv(&mut ctx).await?; let sql = "SELECT c4 FROM aggregate_test_100 WHERE c12 NOT BETWEEN 0 AND 0.995"; let mut actual = execute(&mut ctx, sql).await; actual.sort(); @@ -4550,7 +4553,7 @@ async fn inner_join_nulls() { #[tokio::test] async fn qualified_table_references() -> Result<()> { let mut ctx = ExecutionContext::new(); - register_aggregate_csv(&mut ctx)?; + register_aggregate_csv(&mut ctx).await?; for table_ref in &[ "aggregate_test_100", @@ -4567,7 +4570,7 @@ async fn qualified_table_references() -> Result<()> { #[tokio::test] async fn invalid_qualified_table_references() -> Result<()> { let mut ctx = ExecutionContext::new(); - register_aggregate_csv(&mut ctx)?; + register_aggregate_csv(&mut ctx).await?; for table_ref in &[ "nonexistentschema.aggregate_test_100", @@ -4575,7 +4578,7 @@ async fn invalid_qualified_table_references() -> Result<()> { "way.too.many.namespaces.as.ident.prefixes.aggregate_test_100", ] { let sql = format!("SELECT COUNT(*) FROM {}", table_ref); - assert!(matches!(ctx.sql(&sql), Err(DataFusionError::Plan(_)))); + assert!(matches!(ctx.sql(&sql).await, Err(DataFusionError::Plan(_)))); } Ok(()) } @@ -4651,7 +4654,7 @@ async fn test_random_expression() -> Result<()> { async fn test_cast_expressions_error() -> Result<()> { // sin(utf8) should error let mut ctx = create_ctx()?; - register_aggregate_csv(&mut ctx)?; + register_aggregate_csv(&mut ctx).await?; let sql = "SELECT CAST(c1 AS INT) FROM aggregate_test_100"; let plan = ctx.create_logical_plan(sql).unwrap(); let plan = ctx.optimize(&plan).unwrap(); @@ -4675,7 +4678,7 @@ async fn test_physical_plan_display_indent() { // Hard code target_partitions as it appears in the RepartitionExec output let config = ExecutionConfig::new().with_target_partitions(3); let mut ctx = ExecutionContext::with_config(config); - register_aggregate_csv(&mut ctx).unwrap(); + register_aggregate_csv(&mut ctx).await.unwrap(); let sql = "SELECT c1, MAX(c12), MIN(c12) as the_min \ FROM aggregate_test_100 \ WHERE c12 < 10 \ @@ -4698,7 +4701,7 @@ async fn test_physical_plan_display_indent() { " CoalesceBatchesExec: target_batch_size=4096", " FilterExec: c12@1 < CAST(10 AS Float64)", " RepartitionExec: partitioning=RoundRobinBatch(3)", - " CsvExec: source=Path(ARROW_TEST_DATA/csv/aggregate_test_100.csv: [ARROW_TEST_DATA/csv/aggregate_test_100.csv]), has_header=true", + " CsvExec: files=[ARROW_TEST_DATA/csv/aggregate_test_100.csv], has_header=true, batch_size=8192, limit=None", ]; let data_path = datafusion::test_util::arrow_test_data(); @@ -4722,7 +4725,7 @@ async fn test_physical_plan_display_indent_multi_children() { let config = ExecutionConfig::new().with_target_partitions(3); let mut ctx = ExecutionContext::with_config(config); // ensure indenting works for nodes with multiple children - register_aggregate_csv(&mut ctx).unwrap(); + register_aggregate_csv(&mut ctx).await.unwrap(); let sql = "SELECT c1 \ FROM (select c1 from aggregate_test_100) AS a \ JOIN\ @@ -4743,13 +4746,13 @@ async fn test_physical_plan_display_indent_multi_children() { " ProjectionExec: expr=[c1@0 as c1]", " ProjectionExec: expr=[c1@0 as c1]", " RepartitionExec: partitioning=RoundRobinBatch(3)", - " CsvExec: source=Path(ARROW_TEST_DATA/csv/aggregate_test_100.csv: [ARROW_TEST_DATA/csv/aggregate_test_100.csv]), has_header=true", + " CsvExec: files=[ARROW_TEST_DATA/csv/aggregate_test_100.csv], has_header=true, batch_size=8192, limit=None", " CoalesceBatchesExec: target_batch_size=4096", " RepartitionExec: partitioning=Hash([Column { name: \"c2\", index: 0 }], 3)", " ProjectionExec: expr=[c2@0 as c2]", " ProjectionExec: expr=[c1@0 as c2]", " RepartitionExec: partitioning=RoundRobinBatch(3)", - " CsvExec: source=Path(ARROW_TEST_DATA/csv/aggregate_test_100.csv: [ARROW_TEST_DATA/csv/aggregate_test_100.csv]), has_header=true", + " CsvExec: files=[ARROW_TEST_DATA/csv/aggregate_test_100.csv], has_header=true, batch_size=8192, limit=None", ]; let data_path = datafusion::test_util::arrow_test_data(); @@ -4770,7 +4773,7 @@ async fn test_physical_plan_display_indent_multi_children() { #[tokio::test] async fn test_aggregation_with_bad_arguments() -> Result<()> { let mut ctx = ExecutionContext::new(); - register_aggregate_csv(&mut ctx)?; + register_aggregate_csv(&mut ctx).await?; let sql = "SELECT COUNT(DISTINCT) FROM aggregate_test_100"; let logical_plan = ctx.create_logical_plan(sql)?; let physical_plan = ctx.create_physical_plan(&logical_plan).await; @@ -5001,7 +5004,7 @@ async fn join_tables_with_duplicated_column_name_not_in_on_constraint() -> Resul #[tokio::test] async fn avro_query() { let mut ctx = ExecutionContext::new(); - register_alltypes_avro(&mut ctx); + register_alltypes_avro(&mut ctx).await; // NOTE that string_col is actually a binary column and does not have the UTF8 logical type // so we need an explicit cast let sql = "SELECT id, CAST(string_col AS varchar) FROM alltypes_plain"; @@ -5048,6 +5051,7 @@ async fn avro_query_multiple_files() { table_path.display().to_string().as_str(), AvroReadOptions::default(), ) + .await .unwrap(); // NOTE that string_col is actually a binary column and does not have the UTF8 logical type // so we need an explicit cast @@ -5089,6 +5093,7 @@ async fn avro_single_nan_schema() { &format!("{}/avro/single_nan.avro", testdata), AvroReadOptions::default(), ) + .await .unwrap(); let sql = "SELECT mycol FROM single_nan"; let plan = ctx.create_logical_plan(sql).unwrap(); @@ -5105,7 +5110,7 @@ async fn avro_single_nan_schema() { #[tokio::test] async fn avro_explain() { let mut ctx = ExecutionContext::new(); - register_alltypes_avro(&mut ctx); + register_alltypes_avro(&mut ctx).await; let sql = "EXPLAIN SELECT count(*) from alltypes_plain"; let actual = execute(&mut ctx, sql).await; @@ -5124,7 +5129,7 @@ async fn avro_explain() { \n CoalescePartitionsExec\ \n HashAggregateExec: mode=Partial, gby=[], aggr=[COUNT(UInt8(1))]\ \n RepartitionExec: partitioning=RoundRobinBatch(NUM_CORES)\ - \n AvroExec: source=Path(ARROW_TEST_DATA/avro/alltypes_plain.avro: [ARROW_TEST_DATA/avro/alltypes_plain.avro]), batch_size=8192, limit=None\ + \n AvroExec: files=[ARROW_TEST_DATA/avro/alltypes_plain.avro], batch_size=8192, limit=None\ \n", ], ]; diff --git a/datafusion/tests/statistics.rs b/datafusion/tests/statistics.rs index 7a19aa7deb69..2934d7889215 100644 --- a/datafusion/tests/statistics.rs +++ b/datafusion/tests/statistics.rs @@ -211,7 +211,7 @@ async fn sql_basic() -> Result<()> { let (stats, schema) = fully_defined(); let mut ctx = init_ctx(stats.clone(), schema)?; - let df = ctx.sql("SELECT * from stats_table").unwrap(); + let df = ctx.sql("SELECT * from stats_table").await.unwrap(); let physical_plan = ctx .create_physical_plan(&df.to_logical_plan()) @@ -229,7 +229,10 @@ async fn sql_filter() -> Result<()> { let (stats, schema) = fully_defined(); let mut ctx = init_ctx(stats, schema)?; - let df = ctx.sql("SELECT * FROM stats_table WHERE c1 = 5").unwrap(); + let df = ctx + .sql("SELECT * FROM stats_table WHERE c1 = 5") + .await + .unwrap(); let physical_plan = ctx .create_physical_plan(&df.to_logical_plan()) @@ -247,7 +250,7 @@ async fn sql_limit() -> Result<()> { let (stats, schema) = fully_defined(); let mut ctx = init_ctx(stats.clone(), schema)?; - let df = ctx.sql("SELECT * FROM stats_table LIMIT 5").unwrap(); + let df = ctx.sql("SELECT * FROM stats_table LIMIT 5").await.unwrap(); let physical_plan = ctx .create_physical_plan(&df.to_logical_plan()) .await @@ -263,7 +266,10 @@ async fn sql_limit() -> Result<()> { physical_plan.statistics() ); - let df = ctx.sql("SELECT * FROM stats_table LIMIT 100").unwrap(); + let df = ctx + .sql("SELECT * FROM stats_table LIMIT 100") + .await + .unwrap(); let physical_plan = ctx .create_physical_plan(&df.to_logical_plan()) .await @@ -281,6 +287,7 @@ async fn sql_window() -> Result<()> { let df = ctx .sql("SELECT c2, sum(c1) over (partition by c2) FROM stats_table") + .await .unwrap(); let physical_plan = ctx diff --git a/datafusion/tests/user_defined_plan.rs b/datafusion/tests/user_defined_plan.rs index 27ad901d135d..adb83ac0a202 100644 --- a/datafusion/tests/user_defined_plan.rs +++ b/datafusion/tests/user_defined_plan.rs @@ -91,7 +91,7 @@ use datafusion::logical_plan::DFSchemaRef; /// Execute the specified sql and return the resulting record batches /// pretty printed as a String. async fn exec_sql(ctx: &mut ExecutionContext, sql: &str) -> Result { - let df = ctx.sql(sql)?; + let df = ctx.sql(sql).await?; let batches = df.collect().await?; pretty_format_batches(&batches).map_err(DataFusionError::ArrowError) } @@ -216,9 +216,9 @@ async fn topk_plan() -> Result<()> { let mut ctx = setup_table(make_topk_context()).await?; let expected = vec![ - "| logical_plan after topk | TopK: k=3 |", - "| | Projection: #sales.customer_id, #sales.revenue |", - "| | TableScan: sales projection=Some([0, 1]) |", + "| logical_plan after topk | TopK: k=3 |", + "| | Projection: #sales.customer_id, #sales.revenue |", + "| | TableScan: sales projection=Some([0, 1]) |", ].join("\n"); let explain_query = format!("EXPLAIN VERBOSE {}", QUERY); diff --git a/python/src/context.rs b/python/src/context.rs index 4c47058190d8..24a2cb813045 100644 --- a/python/src/context.rs +++ b/python/src/context.rs @@ -21,6 +21,8 @@ use std::{collections::HashSet, sync::Arc}; use rand::distributions::Alphanumeric; use rand::Rng; +use tokio::runtime::Runtime; + use pyo3::exceptions::PyValueError; use pyo3::prelude::*; @@ -53,11 +55,16 @@ impl ExecutionContext { } /// Returns a DataFrame whose plan corresponds to the SQL statement. - fn sql(&mut self, query: &str) -> PyResult { - let df = self - .ctx - .sql(query) - .map_err(|e| -> errors::DataFusionError { e.into() })?; + fn sql(&mut self, query: &str, py: Python) -> PyResult { + let rt = Runtime::new().unwrap(); + let df = py.allow_threads(|| { + rt.block_on(async { + self.ctx + .sql(query) + .await + .map_err(|e| -> errors::DataFusionError { e.into() }) + }) + })?; Ok(dataframe::DataFrame::new( self.ctx.state.clone(), df.to_logical_plan(), @@ -119,8 +126,13 @@ impl ExecutionContext { Ok(()) } - fn register_parquet(&mut self, name: &str, path: &str) -> PyResult<()> { - errors::wrap(self.ctx.register_parquet(name, path))?; + fn register_parquet(&mut self, name: &str, path: &str, py: Python) -> PyResult<()> { + let rt = Runtime::new().unwrap(); + py.allow_threads(|| { + rt.block_on(async { + errors::wrap(self.ctx.register_parquet(name, path).await) + }) + })?; Ok(()) } @@ -140,6 +152,7 @@ impl ExecutionContext { delimiter: &str, schema_infer_max_records: usize, file_extension: &str, + py: Python, ) -> PyResult<()> { let path = path .to_str() @@ -162,7 +175,12 @@ impl ExecutionContext { .file_extension(file_extension); options.schema = schema.as_ref(); - errors::wrap(self.ctx.register_csv(name, path, options))?; + let rt = Runtime::new().unwrap(); + py.allow_threads(|| { + rt.block_on(async { + errors::wrap(self.ctx.register_csv(name, path, options).await) + }) + })?; Ok(()) } From 47a17fdeca64d6825d5ed0a3749baf61909120f2 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 12 Oct 2021 09:49:04 +0200 Subject: [PATCH 37/39] [doc] clearer doc about why sql() is async --- ballista/rust/client/src/context.rs | 3 ++- ballista/rust/core/proto/ballista.proto | 2 -- datafusion/src/execution/context.rs | 3 ++- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/ballista/rust/client/src/context.rs b/ballista/rust/client/src/context.rs index 615e6c981281..9d5da5ef8a72 100644 --- a/ballista/rust/client/src/context.rs +++ b/ballista/rust/client/src/context.rs @@ -238,7 +238,8 @@ impl BallistaContext { /// Create a DataFrame from a SQL statement. /// - /// Async because CreateExternalTable might require to resolve the schema + /// This method is `async` because queries of type `CREATE EXTERNAL TABLE` + /// might require the schema to be infered. pub async fn sql(&self, sql: &str) -> Result> { let mut ctx = { let state = self.state.lock().unwrap(); diff --git a/ballista/rust/core/proto/ballista.proto b/ballista/rust/core/proto/ballista.proto index 8d9488dc8f49..33638fdc50d4 100644 --- a/ballista/rust/core/proto/ballista.proto +++ b/ballista/rust/core/proto/ballista.proto @@ -20,8 +20,6 @@ syntax = "proto3"; package ballista.protobuf; -import "google/protobuf/timestamp.proto"; - option java_multiple_files = true; option java_package = "org.ballistacompute.protobuf"; option java_outer_classname = "BallistaProto"; diff --git a/datafusion/src/execution/context.rs b/datafusion/src/execution/context.rs index c8331da6f3b7..cc300bba06e6 100644 --- a/datafusion/src/execution/context.rs +++ b/datafusion/src/execution/context.rs @@ -182,7 +182,8 @@ impl ExecutionContext { /// Creates a dataframe that will execute a SQL query. /// - /// async because CreateExternalTable queries might need schema inference + /// This method is `async` because queries of type `CREATE EXTERNAL TABLE` + /// might require the schema to be infered. pub async fn sql(&mut self, sql: &str) -> Result> { let plan = self.create_logical_plan(sql)?; match plan { From 6db3533d390a05efac1e2784fe5587846c728d92 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 12 Oct 2021 11:43:06 +0200 Subject: [PATCH 38/39] [doc] typos and clarity --- ballista/rust/client/src/context.rs | 2 +- .../rust/core/src/serde/physical_plan/to_proto.rs | 4 ++-- datafusion/src/datasource/file_format/mod.rs | 7 +++---- datafusion/src/datasource/file_format/parquet.rs | 2 +- datafusion/src/datasource/listing.rs | 14 +++++++------- datafusion/src/datasource/mod.rs | 3 --- datafusion/src/datasource/object_store/local.rs | 5 +++-- datafusion/src/datasource/object_store/mod.rs | 2 +- datafusion/src/execution/context.rs | 2 +- datafusion/src/physical_plan/expressions/binary.rs | 2 +- 10 files changed, 20 insertions(+), 23 deletions(-) diff --git a/ballista/rust/client/src/context.rs b/ballista/rust/client/src/context.rs index 9d5da5ef8a72..e619f12c1eab 100644 --- a/ballista/rust/client/src/context.rs +++ b/ballista/rust/client/src/context.rs @@ -239,7 +239,7 @@ impl BallistaContext { /// Create a DataFrame from a SQL statement. /// /// This method is `async` because queries of type `CREATE EXTERNAL TABLE` - /// might require the schema to be infered. + /// might require the schema to be inferred. pub async fn sql(&self, sql: &str) -> Result> { let mut ctx = { let state = self.state.lock().unwrap(); diff --git a/ballista/rust/core/src/serde/physical_plan/to_proto.rs b/ballista/rust/core/src/serde/physical_plan/to_proto.rs index c52cbbbb957d..020b6888d8cf 100644 --- a/ballista/rust/core/src/serde/physical_plan/to_proto.rs +++ b/ballista/rust/core/src/serde/physical_plan/to_proto.rs @@ -261,7 +261,7 @@ impl TryInto for Arc { .as_ref() .ok_or_else(|| { BallistaError::General( - "projection in CsvExec dosn not exist.".to_owned(), + "projection in CsvExec does not exist.".to_owned(), ) })? .iter() @@ -320,7 +320,7 @@ impl TryInto for Arc { .as_ref() .ok_or_else(|| { BallistaError::General( - "projection in AvroExec dosn not exist.".to_owned(), + "projection in AvroExec does not exist.".to_owned(), ) })? .iter() diff --git a/datafusion/src/datasource/file_format/mod.rs b/datafusion/src/datasource/file_format/mod.rs index 16d73e63c6a1..d545596f6e5c 100644 --- a/datafusion/src/datasource/file_format/mod.rs +++ b/datafusion/src/datasource/file_format/mod.rs @@ -27,7 +27,6 @@ use std::fmt; use std::sync::Arc; use crate::arrow::datatypes::SchemaRef; -use crate::datasource::{create_max_min_accs, get_col_stats}; use crate::error::Result; use crate::logical_plan::Expr; use crate::physical_plan::{ExecutionPlan, Statistics}; @@ -44,15 +43,15 @@ pub struct PhysicalPlanConfig { pub object_store: Arc, /// Schema before projection pub schema: SchemaRef, - /// Partitioned fields to process in the executor + /// List of files to be processed, grouped into partitions pub files: Vec>, - /// Estimated overall statistics of source plan + /// Estimated overall statistics of the plan, taking `filters` into account 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 + /// The filters that were pushed down to this execution plan pub filters: Vec, /// The minimum number of records required from this source plan pub limit: Option, diff --git a/datafusion/src/datasource/file_format/parquet.rs b/datafusion/src/datasource/file_format/parquet.rs index cd617ebc4c4d..424a2985a3f7 100644 --- a/datafusion/src/datasource/file_format/parquet.rs +++ b/datafusion/src/datasource/file_format/parquet.rs @@ -36,9 +36,9 @@ use parquet::file::statistics::Statistics as ParquetStatistics; use super::FileFormat; 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}; +use crate::datasource::{create_max_min_accs, get_col_stats}; use crate::error::DataFusionError; use crate::error::Result; use crate::logical_plan::combine_filters; diff --git a/datafusion/src/datasource/listing.rs b/datafusion/src/datasource/listing.rs index 959418f4ea02..585a40ffe05e 100644 --- a/datafusion/src/datasource/listing.rs +++ b/datafusion/src/datasource/listing.rs @@ -46,7 +46,7 @@ pub struct ListingOptions { pub file_extension: String, /// The file format pub format: Arc, - /// The expected partition column names. + /// The expected partition column names in the folder structure. /// For example `Vec["a", "b"]` means that the two first levels of /// partitioning expected should be named "a" and "b": /// - If there is a third level of partitioning it will be ignored. @@ -55,11 +55,11 @@ pub struct ListingOptions { /// TODO implement case where partitions.len() > 0 pub partitions: Vec, /// 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. + /// This can add a lot of overhead as it will usually require files + /// to be opened and at least partially parsed. pub collect_stat: bool, - /// Group files to avoid that the number of partitions - /// exceeds this limit + /// Group files to avoid that the number of partitions exceeds + /// this limit pub target_partitions: usize, } @@ -80,8 +80,8 @@ impl ListingOptions { } } - /// Infer the schema of the files at the given uri, including the partitioning - /// columns. + /// Infer the schema of the files at the given path on the provided object store. + /// The inferred schema should include 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 diff --git a/datafusion/src/datasource/mod.rs b/datafusion/src/datasource/mod.rs index 31fde168b046..b607469bff00 100644 --- a/datafusion/src/datasource/mod.rs +++ b/datafusion/src/datasource/mod.rs @@ -40,7 +40,6 @@ use std::pin::Pin; /// if the optional `limit` is provided, includes only sufficient files /// 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 async fn get_statistics_with_limit( all_files: impl Stream>, schema: SchemaRef, @@ -126,7 +125,6 @@ pub async fn get_statistics_with_limit( #[derive(Debug, Clone)] /// A single file that should be read, along with its schema, statistics /// and partition column values that need to be appended to each row. -/// 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_meta: FileMeta, @@ -159,7 +157,6 @@ impl std::fmt::Display for PartitionedFile { #[derive(Debug, Clone)] /// A collection of files that should be read in a single task -/// TODO move back to crate::datasource::mod.rs once legacy cleaned up pub struct FilePartition { /// The index of the partition among all partitions pub index: usize, diff --git a/datafusion/src/datasource/object_store/local.rs b/datafusion/src/datasource/object_store/local.rs index c18af1cd8f44..4f4dbefbca49 100644 --- a/datafusion/src/datasource/object_store/local.rs +++ b/datafusion/src/datasource/object_store/local.rs @@ -152,12 +152,13 @@ async fn list_all(prefix: String) -> Result { } } -/// Create a stream of `ObjectReader` by opening each file in the `files` vector +/// Create a stream of `ObjectReader` by converting each file in the `files` vector +/// into instances of `LocalFileReader` 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 convert a file location to an ObjectReader +/// Helper method to convert a file location to a `LocalFileReader` pub fn local_object_reader(file: String) -> Arc { LocalFileSystem .file_reader(local_file_meta(file).sized_file) diff --git a/datafusion/src/datasource/object_store/mod.rs b/datafusion/src/datasource/object_store/mod.rs index b16684a9db0a..61bc47dc462c 100644 --- a/datafusion/src/datasource/object_store/mod.rs +++ b/datafusion/src/datasource/object_store/mod.rs @@ -73,7 +73,7 @@ pub enum ListEntry { #[derive(Debug, Clone)] pub struct SizedFile { /// Path of the file. It is relative to the current object - /// store (it does not specify the xx:// scheme). + /// store (it does not specify the `xx://` scheme). pub path: String, /// File size in total pub size: u64, diff --git a/datafusion/src/execution/context.rs b/datafusion/src/execution/context.rs index cc300bba06e6..23667f5a6ec8 100644 --- a/datafusion/src/execution/context.rs +++ b/datafusion/src/execution/context.rs @@ -183,7 +183,7 @@ impl ExecutionContext { /// Creates a dataframe that will execute a SQL query. /// /// This method is `async` because queries of type `CREATE EXTERNAL TABLE` - /// might require the schema to be infered. + /// might require the schema to be inferred. pub async fn sql(&mut self, sql: &str) -> Result> { let plan = self.create_logical_plan(sql)?; match plan { diff --git a/datafusion/src/physical_plan/expressions/binary.rs b/datafusion/src/physical_plan/expressions/binary.rs index d58b2ed207a1..5838239eec8c 100644 --- a/datafusion/src/physical_plan/expressions/binary.rs +++ b/datafusion/src/physical_plan/expressions/binary.rs @@ -484,7 +484,7 @@ pub fn binary_operator_data_type( rhs_type: &DataType, ) -> Result { // validate that it is possible to perform the operation on incoming types. - // (or the return datatype cannot be infered) + // (or the return datatype cannot be inferred) let common_type = common_binary_type(lhs_type, op, rhs_type)?; match op { From 2446b8105e8f8177c8b3102863cffdfc5028a463 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 12 Oct 2021 19:30:33 +0200 Subject: [PATCH 39/39] [fix] missing await after rebase --- datafusion/tests/sql.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/tests/sql.rs b/datafusion/tests/sql.rs index 69f1ece3e551..f52920575afb 100644 --- a/datafusion/tests/sql.rs +++ b/datafusion/tests/sql.rs @@ -984,7 +984,7 @@ async fn csv_query_count() -> Result<()> { #[tokio::test] async fn csv_query_approx_count() -> Result<()> { let mut ctx = ExecutionContext::new(); - register_aggregate_csv(&mut ctx)?; + register_aggregate_csv(&mut ctx).await?; let sql = "SELECT approx_distinct(c9) count_c9, approx_distinct(cast(c9 as varchar)) count_c9_str FROM aggregate_test_100"; let actual = execute_to_batches(&mut ctx, sql).await; let expected = vec![