diff --git a/datafusion-examples/examples/parquet_index.rs b/datafusion-examples/examples/parquet_index.rs index 625133ae7cbd..e3387117c91f 100644 --- a/datafusion-examples/examples/parquet_index.rs +++ b/datafusion-examples/examples/parquet_index.rs @@ -25,7 +25,7 @@ use arrow_schema::SchemaRef; use async_trait::async_trait; use datafusion::datasource::listing::PartitionedFile; use datafusion::datasource::physical_plan::{ - parquet::{RequestedStatistics, StatisticsConverter}, + parquet::StatisticsConverter, {FileScanConfig, ParquetExec}, }; use datafusion::datasource::TableProvider; @@ -518,21 +518,17 @@ impl ParquetMetadataIndexBuilder { // extract the parquet statistics from the file's footer let metadata = reader.metadata(); + let row_groups = metadata.row_groups(); // Extract the min/max values for each row group from the statistics - let row_counts = StatisticsConverter::row_counts(reader.metadata())?; - let value_column_mins = StatisticsConverter::try_new( + let converter = StatisticsConverter::try_new( "value", - RequestedStatistics::Min, reader.schema(), - )? - .extract(reader.metadata())?; - let value_column_maxes = StatisticsConverter::try_new( - "value", - RequestedStatistics::Max, - reader.schema(), - )? - .extract(reader.metadata())?; + reader.parquet_schema(), + )?; + let row_counts = StatisticsConverter::row_group_row_counts(row_groups.iter())?; + let value_column_mins = converter.row_group_mins(row_groups.iter())?; + let value_column_maxes = converter.row_group_maxes(row_groups.iter())?; // In a real system you would have to handle nulls, which represent // unknown statistics. All statistics are known in this example diff --git a/datafusion/core/benches/parquet_statistic.rs b/datafusion/core/benches/parquet_statistic.rs index 39ed39d49e38..5fd6b0066eb2 100644 --- a/datafusion/core/benches/parquet_statistic.rs +++ b/datafusion/core/benches/parquet_statistic.rs @@ -24,9 +24,7 @@ use arrow_schema::{ Field, Schema, }; use criterion::{criterion_group, criterion_main, BenchmarkId, Criterion}; -use datafusion::datasource::physical_plan::parquet::{ - RequestedStatistics, StatisticsConverter, -}; +use datafusion::datasource::physical_plan::parquet::StatisticsConverter; use parquet::arrow::{arrow_reader::ArrowReaderBuilder, ArrowWriter}; use parquet::file::properties::WriterProperties; use std::sync::Arc; @@ -159,6 +157,7 @@ fn criterion_benchmark(c: &mut Criterion) { let file = file.reopen().unwrap(); let reader = ArrowReaderBuilder::try_new(file).unwrap(); let metadata = reader.metadata(); + let row_groups = metadata.row_groups(); let mut group = c.benchmark_group(format!("Extract statistics for {}", dtype.clone())); @@ -166,34 +165,18 @@ fn criterion_benchmark(c: &mut Criterion) { BenchmarkId::new("extract_statistics", dtype.clone()), |b| { b.iter(|| { - let _ = StatisticsConverter::try_new( - "col", - RequestedStatistics::Min, - reader.schema(), - ) - .unwrap() - .extract(metadata) - .unwrap(); - - let _ = StatisticsConverter::try_new( - "col", - RequestedStatistics::Max, - reader.schema(), - ) - .unwrap() - .extract(reader.metadata()) - .unwrap(); - - let _ = StatisticsConverter::try_new( + let converter = StatisticsConverter::try_new( "col", - RequestedStatistics::NullCount, reader.schema(), + reader.parquet_schema(), ) - .unwrap() - .extract(reader.metadata()) .unwrap(); - let _ = StatisticsConverter::row_counts(reader.metadata()).unwrap(); + let _ = converter.row_group_mins(row_groups.iter()).unwrap(); + let _ = converter.row_group_maxes(row_groups.iter()).unwrap(); + let _ = converter.row_group_null_counts(row_groups.iter()).unwrap(); + let _ = StatisticsConverter::row_group_row_counts(row_groups.iter()) + .unwrap(); }) }, ); diff --git a/datafusion/core/src/datasource/physical_plan/parquet/metrics.rs b/datafusion/core/src/datasource/physical_plan/parquet/metrics.rs index c2a7e4345a5b..3599a2f5fa51 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/metrics.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/metrics.rs @@ -29,11 +29,11 @@ use crate::physical_plan::metrics::{ pub struct ParquetFileMetrics { /// Number of times the predicate could not be evaluated pub predicate_evaluation_errors: Count, - /// Number of row groups whose bloom filters were checked and matched + /// Number of row groups whose bloom filters were checked and matched (not pruned) pub row_groups_matched_bloom_filter: Count, /// Number of row groups pruned by bloom filters pub row_groups_pruned_bloom_filter: Count, - /// Number of row groups whose statistics were checked and matched + /// Number of row groups whose statistics were checked and matched (not pruned) pub row_groups_matched_statistics: Count, /// Number of row groups pruned by statistics pub row_groups_pruned_statistics: Count, diff --git a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs index 04b25069e923..39c8761eac45 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs @@ -64,7 +64,7 @@ pub use access_plan::{ParquetAccessPlan, RowGroupAccess}; pub use metrics::ParquetFileMetrics; use opener::ParquetOpener; pub use reader::{DefaultParquetFileReaderFactory, ParquetFileReaderFactory}; -pub use statistics::{RequestedStatistics, StatisticsConverter}; +pub use statistics::StatisticsConverter; pub use writer::plan_to_parquet; /// Execution plan for reading one or more Parquet files. diff --git a/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs b/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs index 20656634c472..06eb8f79dada 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs @@ -17,11 +17,9 @@ use arrow::{array::ArrayRef, datatypes::Schema}; use arrow_array::BooleanArray; -use arrow_schema::FieldRef; -use datafusion_common::{Column, ScalarValue}; +use datafusion_common::{Column, Result, ScalarValue}; use parquet::basic::Type; use parquet::data_type::Decimal; -use parquet::file::metadata::ColumnChunkMetaData; use parquet::schema::types::SchemaDescriptor; use parquet::{ arrow::{async_reader::AsyncFileReader, ParquetRecordBatchStreamBuilder}, @@ -29,14 +27,13 @@ use parquet::{ file::metadata::RowGroupMetaData, }; use std::collections::{HashMap, HashSet}; +use std::sync::Arc; use crate::datasource::listing::FileRange; -use crate::datasource::physical_plan::parquet::statistics::{ - max_statistics, min_statistics, parquet_column, -}; +use crate::datasource::physical_plan::parquet::statistics::parquet_column; use crate::physical_optimizer::pruning::{PruningPredicate, PruningStatistics}; -use super::{ParquetAccessPlan, ParquetFileMetrics}; +use super::{ParquetAccessPlan, ParquetFileMetrics, StatisticsConverter}; /// Reduces the [`ParquetAccessPlan`] based on row group level metadata. /// @@ -113,32 +110,37 @@ impl RowGroupAccessPlanFilter { metrics: &ParquetFileMetrics, ) { assert_eq!(groups.len(), self.access_plan.len()); - for (idx, metadata) in groups.iter().enumerate() { - if !self.access_plan.should_scan(idx) { - continue; - } - let pruning_stats = RowGroupPruningStatistics { - parquet_schema, - row_group_metadata: metadata, - arrow_schema, - }; - match predicate.prune(&pruning_stats) { - Ok(values) => { - // NB: false means don't scan row group - if !values[0] { + // Indexes of row groups still to scan + let row_group_indexes = self.access_plan.row_group_indexes(); + let row_group_metadatas = row_group_indexes + .iter() + .map(|&i| &groups[i]) + .collect::>(); + + let pruning_stats = RowGroupPruningStatistics { + parquet_schema, + row_group_metadatas, + arrow_schema, + }; + + // try to prune the row groups in a single call + match predicate.prune(&pruning_stats) { + Ok(values) => { + // values[i] is false means the predicate could not be true for row group i + for (idx, &value) in row_group_indexes.iter().zip(values.iter()) { + if !value { + self.access_plan.skip(*idx); metrics.row_groups_pruned_statistics.add(1); - self.access_plan.skip(idx); - continue; + } else { + metrics.row_groups_matched_statistics.add(1); } } - // stats filter array could not be built - // don't prune this row group - Err(e) => { - log::debug!("Error evaluating row group predicate values {e}"); - metrics.predicate_evaluation_errors.add(1); - } } - metrics.row_groups_matched_statistics.add(1); + // stats filter array could not be built, so we can't prune + Err(e) => { + log::debug!("Error evaluating row group predicate values {e}"); + metrics.predicate_evaluation_errors.add(1); + } } } @@ -337,49 +339,55 @@ impl PruningStatistics for BloomFilterStatistics { } } -/// Wraps [`RowGroupMetaData`] in a way that implements [`PruningStatistics`] -/// -/// Note: This should be implemented for an array of [`RowGroupMetaData`] instead -/// of per row-group +/// Wraps a slice of [`RowGroupMetaData`] in a way that implements [`PruningStatistics`] struct RowGroupPruningStatistics<'a> { parquet_schema: &'a SchemaDescriptor, - row_group_metadata: &'a RowGroupMetaData, + row_group_metadatas: Vec<&'a RowGroupMetaData>, arrow_schema: &'a Schema, } impl<'a> RowGroupPruningStatistics<'a> { - /// Lookups up the parquet column by name - fn column(&self, name: &str) -> Option<(&ColumnChunkMetaData, &FieldRef)> { - let (idx, field) = parquet_column(self.parquet_schema, self.arrow_schema, name)?; - Some((self.row_group_metadata.column(idx), field)) + /// Return an iterator over the row group metadata + fn metadata_iter(&'a self) -> impl Iterator + 'a { + self.row_group_metadatas.iter().copied() + } + + fn statistics_converter<'b>( + &'a self, + column: &'b Column, + ) -> Result> { + StatisticsConverter::try_new(&column.name, self.arrow_schema, self.parquet_schema) } } impl<'a> PruningStatistics for RowGroupPruningStatistics<'a> { fn min_values(&self, column: &Column) -> Option { - let (column, field) = self.column(&column.name)?; - min_statistics(field.data_type(), std::iter::once(column.statistics())).ok() + self.statistics_converter(column) + .and_then(|c| c.row_group_mins(self.metadata_iter())) + .ok() } fn max_values(&self, column: &Column) -> Option { - let (column, field) = self.column(&column.name)?; - max_statistics(field.data_type(), std::iter::once(column.statistics())).ok() + self.statistics_converter(column) + .and_then(|c| c.row_group_maxes(self.metadata_iter())) + .ok() } fn num_containers(&self) -> usize { - 1 + self.row_group_metadatas.len() } fn null_counts(&self, column: &Column) -> Option { - let (c, _) = self.column(&column.name)?; - let scalar = ScalarValue::UInt64(Some(c.statistics()?.null_count())); - scalar.to_array().ok() + self.statistics_converter(column) + .and_then(|c| c.row_group_null_counts(self.metadata_iter())) + .ok() } - fn row_counts(&self, column: &Column) -> Option { - let (c, _) = self.column(&column.name)?; - let scalar = ScalarValue::UInt64(Some(c.num_values() as u64)); - scalar.to_array().ok() + fn row_counts(&self, _column: &Column) -> Option { + // row counts are the same for all columns in a row group + StatisticsConverter::row_group_row_counts(self.metadata_iter()) + .ok() + .map(|counts| Arc::new(counts) as ArrayRef) } fn contained( @@ -406,6 +414,7 @@ mod tests { use parquet::arrow::async_reader::ParquetObjectReader; use parquet::basic::LogicalType; use parquet::data_type::{ByteArray, FixedLenByteArray}; + use parquet::file::metadata::ColumnChunkMetaData; use parquet::{ basic::Type as PhysicalType, file::statistics::Statistics as ParquetStatistics, schema::types::SchemaDescPtr, diff --git a/datafusion/core/src/datasource/physical_plan/parquet/statistics.rs b/datafusion/core/src/datasource/physical_plan/parquet/statistics.rs index a73538d02a7f..a4a919f20d0f 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/statistics.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/statistics.rs @@ -33,7 +33,7 @@ use arrow_array::{ use arrow_schema::{Field, FieldRef, Schema, TimeUnit}; use datafusion_common::{internal_datafusion_err, internal_err, plan_err, Result}; use half::f16; -use parquet::file::metadata::ParquetMetaData; +use parquet::file::metadata::RowGroupMetaData; use parquet::file::statistics::Statistics as ParquetStatistics; use parquet::schema::types::SchemaDescriptor; use paste::paste; @@ -558,17 +558,6 @@ pub(crate) fn max_statistics<'a, I: Iterator { - /// The name of the column to extract statistics for - column_name: &'a str, - /// The type of statistics to extract - statistics_type: RequestedStatistics, - /// The arrow schema of the query - arrow_schema: &'a Schema, + /// the index of the matched column in the parquet schema + parquet_index: Option, /// The field (with data type) of the column in the arrow schema arrow_field: &'a Field, } impl<'a> StatisticsConverter<'a> { - /// Returns a [`UInt64Array`] with counts for each row group + /// Returns a [`UInt64Array`] with row counts for each row group + /// + /// # Return Value /// /// The returned array has no nulls, and has one value for each row group. /// Each value is the number of rows in the row group. - pub fn row_counts(metadata: &ParquetMetaData) -> Result { - let row_groups = metadata.row_groups(); - let mut builder = UInt64Array::builder(row_groups.len()); - for row_group in row_groups { - let row_count = row_group.num_rows(); + /// + /// # Example + /// ```no_run + /// # use parquet::file::metadata::ParquetMetaData; + /// # use datafusion::datasource::physical_plan::parquet::StatisticsConverter; + /// # fn get_parquet_metadata() -> ParquetMetaData { unimplemented!() } + /// // Given the metadata for a parquet file + /// let metadata: ParquetMetaData = get_parquet_metadata(); + /// // get the row counts for each row group + /// let row_counts = StatisticsConverter::row_group_row_counts(metadata + /// .row_groups() + /// .iter() + /// ); + /// ``` + pub fn row_group_row_counts(metadatas: I) -> Result + where + I: IntoIterator, + { + let mut builder = UInt64Array::builder(10); + for metadata in metadatas.into_iter() { + let row_count = metadata.num_rows(); let row_count: u64 = row_count.try_into().map_err(|e| { internal_datafusion_err!( "Parquet row count {row_count} too large to convert to u64: {e}" @@ -626,11 +621,21 @@ impl<'a> StatisticsConverter<'a> { Ok(builder.finish()) } - /// create an new statistics converter - pub fn try_new( - column_name: &'a str, - statistics_type: RequestedStatistics, + /// Create a new `StatisticsConverter` to extract statistics for a column + /// + /// Note if there is no corresponding column in the parquet file, the returned + /// arrays will be null. This can happen if the column is in the arrow + /// schema but not in the parquet schema due to schema evolution. + /// + /// See example on [`Self::row_group_mins`] for usage + /// + /// # Errors + /// + /// * If the column is not found in the arrow schema + pub fn try_new<'b>( + column_name: &'b str, arrow_schema: &'a Schema, + parquet_schema: &'a SchemaDescriptor, ) -> Result { // ensure the requested column is in the arrow schema let Some((_idx, arrow_field)) = arrow_schema.column_with_name(column_name) else { @@ -639,67 +644,136 @@ impl<'a> StatisticsConverter<'a> { column_name ); }; - Ok(Self { - column_name, - statistics_type, + + // find the column in the parquet schema, if not, return a null array + let parquet_index = match parquet_column( + parquet_schema, arrow_schema, + column_name, + ) { + Some((parquet_idx, matched_field)) => { + // sanity check that matching field matches the arrow field + if matched_field.as_ref() != arrow_field { + return internal_err!( + "Matched column '{:?}' does not match original matched column '{:?}'", + matched_field, + arrow_field + ); + } + Some(parquet_idx) + } + None => None, + }; + + Ok(Self { + parquet_index, arrow_field, }) } - /// extract the statistics from a parquet file, given the parquet file's metadata + /// Extract the minimum values from row group statistics in [`RowGroupMetaData`] + /// + /// # Return Value /// - /// The returned array contains 1 value for each row group in the parquet - /// file in order + /// The returned array contains 1 value for each row group, in the same order as `metadatas` /// /// Each value is either - /// * the requested statistics type for the column + /// * the minimum value for the column /// * a null value, if the statistics can not be extracted /// - /// Note that a null value does NOT mean the min or max value was actually + /// Note that a null value does NOT mean the min value was actually /// `null` it means it the requested statistic is unknown /// + /// # Errors + /// /// Reasons for not being able to extract the statistics include: /// * the column is not present in the parquet file /// * statistics for the column are not present in the row group /// * the stored statistic value can not be converted to the requested type - pub fn extract(&self, metadata: &ParquetMetaData) -> Result { + /// + /// # Example + /// ```no_run + /// # use arrow::datatypes::Schema; + /// # use arrow_array::ArrayRef; + /// # use parquet::file::metadata::ParquetMetaData; + /// # use datafusion::datasource::physical_plan::parquet::StatisticsConverter; + /// # fn get_parquet_metadata() -> ParquetMetaData { unimplemented!() } + /// # fn get_arrow_schema() -> Schema { unimplemented!() } + /// // Given the metadata for a parquet file and the arrow schema + /// let metadata: ParquetMetaData = get_parquet_metadata(); + /// let arrow_schema: Schema = get_arrow_schema(); + /// let parquet_schema = metadata.file_metadata().schema_descr(); + /// // create a converter + /// let converter = StatisticsConverter::try_new("foo", &arrow_schema, parquet_schema) + /// .unwrap(); + /// // get the minimum value for the column "foo" in the parquet file + /// let min_values: ArrayRef = converter + /// .row_group_mins(metadata.row_groups().iter()) + /// .unwrap(); + /// ``` + pub fn row_group_mins(&self, metadatas: I) -> Result + where + I: IntoIterator, + { let data_type = self.arrow_field.data_type(); - let num_row_groups = metadata.row_groups().len(); - let parquet_schema = metadata.file_metadata().schema_descr(); - let row_groups = metadata.row_groups(); + let Some(parquet_index) = self.parquet_index else { + return Ok(self.make_null_array(data_type, metadatas)); + }; - // find the column in the parquet schema, if not, return a null array - let Some((parquet_idx, matched_field)) = - parquet_column(parquet_schema, self.arrow_schema, self.column_name) - else { - // column was in the arrow schema but not in the parquet schema, so return a null array - return Ok(new_null_array(data_type, num_row_groups)); + let iter = metadatas + .into_iter() + .map(|x| x.column(parquet_index).statistics()); + min_statistics(data_type, iter) + } + + /// Extract the maximum values from row group statistics in [`RowGroupMetaData`] + /// + /// See docs on [`Self::row_group_mins`] for details + pub fn row_group_maxes(&self, metadatas: I) -> Result + where + I: IntoIterator, + { + let data_type = self.arrow_field.data_type(); + + let Some(parquet_index) = self.parquet_index else { + return Ok(self.make_null_array(data_type, metadatas)); }; - // sanity check that matching field matches the arrow field - if matched_field.as_ref() != self.arrow_field { - return internal_err!( - "Matched column '{:?}' does not match original matched column '{:?}'", - matched_field, - self.arrow_field - ); - } + let iter = metadatas + .into_iter() + .map(|x| x.column(parquet_index).statistics()); + max_statistics(data_type, iter) + } - // Get an iterator over the column statistics - let iter = row_groups - .iter() - .map(|x| x.column(parquet_idx).statistics()); - - match self.statistics_type { - RequestedStatistics::Min => min_statistics(data_type, iter), - RequestedStatistics::Max => max_statistics(data_type, iter), - RequestedStatistics::NullCount => { - let null_counts = iter.map(|stats| stats.map(|s| s.null_count())); - Ok(Arc::new(UInt64Array::from_iter(null_counts))) - } - } + /// Extract the null counts from row group statistics in [`RowGroupMetaData`] + /// + /// See docs on [`Self::row_group_mins`] for details + pub fn row_group_null_counts(&self, metadatas: I) -> Result + where + I: IntoIterator, + { + let data_type = self.arrow_field.data_type(); + + let Some(parquet_index) = self.parquet_index else { + return Ok(self.make_null_array(data_type, metadatas)); + }; + + let null_counts = metadatas + .into_iter() + .map(|x| x.column(parquet_index).statistics()) + .map(|s| s.map(|s| s.null_count())); + Ok(Arc::new(UInt64Array::from_iter(null_counts))) + } + + /// Returns a null array of data_type with one element per row group + fn make_null_array(&self, data_type: &DataType, metadatas: I) -> ArrayRef + where + I: IntoIterator, + { + // column was in the arrow schema but not in the parquet schema, so return a null array + let num_row_groups = metadatas.into_iter().count(); + new_null_array(data_type, num_row_groups) } } diff --git a/datafusion/core/tests/parquet/arrow_statistics.rs b/datafusion/core/tests/parquet/arrow_statistics.rs index b378b2a6c3df..0e23e6824027 100644 --- a/datafusion/core/tests/parquet/arrow_statistics.rs +++ b/datafusion/core/tests/parquet/arrow_statistics.rs @@ -38,9 +38,7 @@ use arrow_array::{ TimestampSecondArray, UInt16Array, UInt32Array, UInt64Array, UInt8Array, }; use arrow_schema::{DataType, Field, Schema}; -use datafusion::datasource::physical_plan::parquet::{ - RequestedStatistics, StatisticsConverter, -}; +use datafusion::datasource::physical_plan::parquet::StatisticsConverter; use half::f16; use parquet::arrow::arrow_reader::{ArrowReaderBuilder, ParquetRecordBatchReaderBuilder}; use parquet::arrow::ArrowWriter; @@ -188,41 +186,28 @@ impl<'a> Test<'a> { column_name, } = self; - let min = StatisticsConverter::try_new( + let converter = StatisticsConverter::try_new( column_name, - RequestedStatistics::Min, reader.schema(), + reader.parquet_schema(), ) - .unwrap() - .extract(reader.metadata()) .unwrap(); + let row_groups = reader.metadata().row_groups(); + let min = converter.row_group_mins(row_groups).unwrap(); + assert_eq!( &min, &expected_min, "{column_name}: Mismatch with expected minimums" ); - let max = StatisticsConverter::try_new( - column_name, - RequestedStatistics::Max, - reader.schema(), - ) - .unwrap() - .extract(reader.metadata()) - .unwrap(); + let max = converter.row_group_maxes(row_groups).unwrap(); assert_eq!( &max, &expected_max, "{column_name}: Mismatch with expected maximum" ); - let null_counts = StatisticsConverter::try_new( - column_name, - RequestedStatistics::NullCount, - reader.schema(), - ) - .unwrap() - .extract(reader.metadata()) - .unwrap(); + let null_counts = converter.row_group_null_counts(row_groups).unwrap(); let expected_null_counts = Arc::new(expected_null_counts) as ArrayRef; assert_eq!( &null_counts, &expected_null_counts, @@ -230,7 +215,10 @@ impl<'a> Test<'a> { Actual: {null_counts:?}. Expected: {expected_null_counts:?}" ); - let row_counts = StatisticsConverter::row_counts(reader.metadata()).unwrap(); + let row_counts = StatisticsConverter::row_group_row_counts( + reader.metadata().row_groups().iter(), + ) + .unwrap(); assert_eq!( row_counts, expected_row_counts, "{column_name}: Mismatch with expected row counts. \ @@ -249,13 +237,13 @@ impl<'a> Test<'a> { column_name, } = self; - let min = StatisticsConverter::try_new( + let converter = StatisticsConverter::try_new( column_name, - RequestedStatistics::Min, reader.schema(), + reader.parquet_schema(), ); - assert!(min.is_err()); + assert!(converter.is_err()); } } diff --git a/datafusion/core/tests/parquet/row_group_pruning.rs b/datafusion/core/tests/parquet/row_group_pruning.rs index 1a174a325bd5..536ac5414a9a 100644 --- a/datafusion/core/tests/parquet/row_group_pruning.rs +++ b/datafusion/core/tests/parquet/row_group_pruning.rs @@ -109,7 +109,7 @@ impl RowGroupPruningTest { assert_eq!( output.predicate_evaluation_errors(), self.expected_errors, - "mismatched predicate_evaluation" + "mismatched predicate_evaluation error" ); assert_eq!( output.row_groups_matched_statistics(),