-
Notifications
You must be signed in to change notification settings - Fork 1.8k
Handle merged schemas in parquet pruning #2170
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
9e5c51d
04d9dec
067c10d
adc0a28
a0809b5
869fe27
5e6f77d
b1aaaa8
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -25,6 +25,7 @@ use arrow::datatypes::Schema; | |
| use arrow::datatypes::SchemaRef; | ||
| use async_trait::async_trait; | ||
| use futures::TryStreamExt; | ||
| use hashbrown::HashMap; | ||
| use parquet::arrow::ArrowReader; | ||
| use parquet::arrow::ParquetFileArrowReader; | ||
| use parquet::errors::ParquetError; | ||
|
|
@@ -46,7 +47,7 @@ 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::file_format::{ParquetExec, SchemaAdapter}; | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. 👍 |
||
| use crate::physical_plan::ExecutionPlan; | ||
| use crate::physical_plan::{Accumulator, Statistics}; | ||
| use datafusion_data_access::object_store::{ObjectReader, ObjectReaderStream}; | ||
|
|
@@ -99,8 +100,12 @@ impl FileFormat for ParquetFormat { | |
| Ok(Arc::new(merged_schema)) | ||
| } | ||
|
|
||
| async fn infer_stats(&self, reader: Arc<dyn ObjectReader>) -> Result<Statistics> { | ||
| let stats = fetch_statistics(reader)?; | ||
| async fn infer_stats( | ||
| &self, | ||
| reader: Arc<dyn ObjectReader>, | ||
| table_schema: SchemaRef, | ||
| ) -> Result<Statistics> { | ||
| let stats = fetch_statistics(reader, table_schema)?; | ||
| Ok(stats) | ||
| } | ||
|
|
||
|
|
@@ -279,46 +284,65 @@ fn fetch_schema(object_reader: Arc<dyn ObjectReader>) -> Result<Schema> { | |
| } | ||
|
|
||
| /// Read and parse the statistics of the Parquet file at location `path` | ||
| fn fetch_statistics(object_reader: Arc<dyn ObjectReader>) -> Result<Statistics> { | ||
| fn fetch_statistics( | ||
| object_reader: Arc<dyn ObjectReader>, | ||
| table_schema: SchemaRef, | ||
| ) -> Result<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()?; | ||
| let num_fields = schema.fields().len(); | ||
| let fields = schema.fields().to_vec(); | ||
| let file_schema = arrow_reader.get_schema()?; | ||
| let num_fields = table_schema.fields().len(); | ||
| let fields = table_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); | ||
| let schema_adapter = SchemaAdapter::new(table_schema.clone()); | ||
|
|
||
| let (mut max_values, mut min_values) = create_max_min_accs(&table_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 | ||
| } | ||
| let mut column_stats: HashMap<usize, (u64, &ParquetStatistics)> = HashMap::new(); | ||
|
|
||
| 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) | ||
| column_stats.insert(i, (stat.null_count(), stat)); | ||
| } | ||
| } | ||
|
|
||
| if has_statistics { | ||
| for (table_idx, null_cnt) in null_counts.iter_mut().enumerate() { | ||
| if let Some(file_idx) = | ||
| schema_adapter.map_column_index(table_idx, &file_schema) | ||
| { | ||
| if let Some((null_count, stats)) = column_stats.get(&file_idx) { | ||
| *null_cnt += *null_count as usize; | ||
| summarize_min_max( | ||
| &mut max_values, | ||
| &mut min_values, | ||
| &fields, | ||
| table_idx, | ||
| stats, | ||
| ) | ||
| } | ||
| } else { | ||
| *null_cnt += num_rows as usize; | ||
| } | ||
| } | ||
| } | ||
| } | ||
|
|
||
| let column_stats = if has_statistics { | ||
| Some(get_col_stats( | ||
| &schema, | ||
| &table_schema, | ||
| null_counts, | ||
| &mut max_values, | ||
| &mut min_values, | ||
|
|
@@ -369,10 +393,102 @@ mod tests { | |
|
|
||
| use crate::prelude::{SessionConfig, SessionContext}; | ||
| use arrow::array::{ | ||
| BinaryArray, BooleanArray, Float32Array, Float64Array, Int32Array, | ||
| TimestampNanosecondArray, | ||
| ArrayRef, BinaryArray, BooleanArray, Float32Array, Float64Array, Int32Array, | ||
| StringArray, TimestampNanosecondArray, | ||
| }; | ||
| use arrow::record_batch::RecordBatch; | ||
| use datafusion_common::ScalarValue; | ||
| use futures::StreamExt; | ||
| use parquet::arrow::ArrowWriter; | ||
| use parquet::file::properties::WriterProperties; | ||
| use tempfile::NamedTempFile; | ||
|
|
||
| // Add a new column with the specified field name to the RecordBatch | ||
| fn add_to_batch( | ||
| batch: &RecordBatch, | ||
| field_name: &str, | ||
| array: ArrayRef, | ||
| ) -> RecordBatch { | ||
| let mut fields = batch.schema().fields().clone(); | ||
| fields.push(Field::new(field_name, array.data_type().clone(), true)); | ||
| let schema = Arc::new(Schema::new(fields)); | ||
|
|
||
| let mut columns = batch.columns().to_vec(); | ||
| columns.push(array); | ||
| RecordBatch::try_new(schema, columns).expect("error; creating record batch") | ||
| } | ||
|
|
||
| fn create_batch(columns: Vec<(&str, ArrayRef)>) -> RecordBatch { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This looks very similar / the same as |
||
| columns.into_iter().fold( | ||
| RecordBatch::new_empty(Arc::new(Schema::new(vec![]))), | ||
| |batch, (field_name, arr)| add_to_batch(&batch, field_name, arr.clone()), | ||
| ) | ||
| } | ||
|
|
||
| async fn create_table( | ||
| batches: Vec<RecordBatch>, | ||
| ) -> Result<(Vec<NamedTempFile>, Schema)> { | ||
| let merged_schema = | ||
| Schema::try_merge(batches.iter().map(|b| b.schema().as_ref().clone()))?; | ||
|
|
||
| let files: Vec<_> = batches | ||
| .into_iter() | ||
| .map(|batch| { | ||
| let output = tempfile::NamedTempFile::new().expect("creating temp file"); | ||
|
|
||
| let props = WriterProperties::builder().build(); | ||
| let file: std::fs::File = (*output.as_file()) | ||
| .try_clone() | ||
| .expect("cloning file descriptor"); | ||
| let mut writer = ArrowWriter::try_new(file, batch.schema(), Some(props)) | ||
| .expect("creating writer"); | ||
|
|
||
| writer.write(&batch).expect("Writing batch"); | ||
| writer.close().unwrap(); | ||
| output | ||
| }) | ||
| .collect(); | ||
|
|
||
| Ok((files, merged_schema)) | ||
| } | ||
|
|
||
| #[tokio::test] | ||
| async fn read_merged_batches() -> Result<()> { | ||
| let c1: ArrayRef = | ||
| Arc::new(StringArray::from(vec![Some("Foo"), None, Some("bar")])); | ||
|
|
||
| let c2: ArrayRef = Arc::new(Int64Array::from(vec![Some(1), Some(2), None])); | ||
|
|
||
| let batch1 = create_batch(vec![("c1", c1.clone())]); | ||
|
|
||
| let batch2 = create_batch(vec![("c2", c2)]); | ||
|
|
||
| let (files, schema) = create_table(vec![batch1, batch2]).await?; | ||
| let table_schema = Arc::new(schema); | ||
|
|
||
| let reader = local_object_reader(files[0].path().to_string_lossy().to_string()); | ||
|
|
||
| let stats = fetch_statistics(reader, table_schema.clone())?; | ||
|
|
||
| assert_eq!(stats.num_rows, Some(3)); | ||
| let c1_stats = &stats.column_statistics.as_ref().expect("missing c1 stats")[0]; | ||
| let c2_stats = &stats.column_statistics.as_ref().expect("missing c2 stats")[1]; | ||
| assert_eq!(c1_stats.null_count, Some(1)); | ||
| assert_eq!(c2_stats.null_count, Some(3)); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. this is cool to fill in the null stats for the missing column 👍 |
||
|
|
||
| let reader = local_object_reader(files[1].path().to_string_lossy().to_string()); | ||
|
|
||
| let stats = fetch_statistics(reader, table_schema)?; | ||
| assert_eq!(stats.num_rows, Some(3)); | ||
| let c1_stats = &stats.column_statistics.as_ref().expect("missing c1 stats")[0]; | ||
| let c2_stats = &stats.column_statistics.as_ref().expect("missing c2 stats")[1]; | ||
| assert_eq!(c1_stats.null_count, Some(3)); | ||
| assert_eq!(c2_stats.null_count, Some(1)); | ||
| assert_eq!(c2_stats.max_value, Some(ScalarValue::Int64(Some(2)))); | ||
| assert_eq!(c2_stats.min_value, Some(ScalarValue::Int64(Some(1)))); | ||
|
|
||
| Ok(()) | ||
| } | ||
|
|
||
| #[tokio::test] | ||
| async fn read_small_batches() -> Result<()> { | ||
|
|
@@ -645,7 +761,7 @@ mod tests { | |
| .await | ||
| .expect("Schema inference"); | ||
| let statistics = format | ||
| .infer_stats(local_object_reader(filename.clone())) | ||
| .infer_stats(local_object_reader(filename.clone()), file_schema.clone()) | ||
| .await | ||
| .expect("Stats inference"); | ||
| let file_groups = vec![vec![local_unpartitioned_file(filename.clone())]]; | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.