Skip to content

Commit 70256ba

Browse files
feat: Update Parquet row filtering to handle type coercion (#10716)
* test: Add a failing test to show the lack of type coercion in row filters * feat: update parquet row filter to handle type coercion * chore: lint/fmt * chore: test improvements and cleanup
1 parent df5dab7 commit 70256ba

File tree

3 files changed

+156
-11
lines changed

3 files changed

+156
-11
lines changed

datafusion/core/src/datasource/physical_plan/parquet/opener.rs

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -114,6 +114,7 @@ impl FileOpener for ParquetOpener {
114114
builder.metadata(),
115115
reorder_predicates,
116116
&file_metrics,
117+
Arc::clone(&schema_mapping),
117118
);
118119

119120
match row_filter {

datafusion/core/src/datasource/physical_plan/parquet/row_filter.rs

Lines changed: 110 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -18,13 +18,15 @@
1818
use std::collections::BTreeSet;
1919
use std::sync::Arc;
2020

21-
use super::ParquetFileMetrics;
22-
use crate::physical_plan::metrics;
23-
2421
use arrow::array::BooleanArray;
2522
use arrow::datatypes::{DataType, Schema};
2623
use arrow::error::{ArrowError, Result as ArrowResult};
2724
use arrow::record_batch::RecordBatch;
25+
use parquet::arrow::arrow_reader::{ArrowPredicate, RowFilter};
26+
use parquet::arrow::ProjectionMask;
27+
use parquet::file::metadata::ParquetMetaData;
28+
29+
use crate::datasource::schema_adapter::SchemaMapper;
2830
use datafusion_common::cast::as_boolean_array;
2931
use datafusion_common::tree_node::{
3032
Transformed, TransformedResult, TreeNode, TreeNodeRecursion, TreeNodeRewriter,
@@ -34,9 +36,9 @@ use datafusion_physical_expr::expressions::{Column, Literal};
3436
use datafusion_physical_expr::utils::reassign_predicate_columns;
3537
use datafusion_physical_expr::{split_conjunction, PhysicalExpr};
3638

37-
use parquet::arrow::arrow_reader::{ArrowPredicate, RowFilter};
38-
use parquet::arrow::ProjectionMask;
39-
use parquet::file::metadata::ParquetMetaData;
39+
use crate::physical_plan::metrics;
40+
41+
use super::ParquetFileMetrics;
4042

4143
/// This module contains utilities for enabling the pushdown of DataFusion filter predicates (which
4244
/// can be any DataFusion `Expr` that evaluates to a `BooleanArray`) to the parquet decoder level in `arrow-rs`.
@@ -78,6 +80,8 @@ pub(crate) struct DatafusionArrowPredicate {
7880
rows_filtered: metrics::Count,
7981
/// how long was spent evaluating this predicate
8082
time: metrics::Time,
83+
/// used to perform type coercion while filtering rows
84+
schema_mapping: Arc<dyn SchemaMapper>,
8185
}
8286

8387
impl DatafusionArrowPredicate {
@@ -87,6 +91,7 @@ impl DatafusionArrowPredicate {
8791
metadata: &ParquetMetaData,
8892
rows_filtered: metrics::Count,
8993
time: metrics::Time,
94+
schema_mapping: Arc<dyn SchemaMapper>,
9095
) -> Result<Self> {
9196
let schema = Arc::new(schema.project(&candidate.projection)?);
9297
let physical_expr = reassign_predicate_columns(candidate.expr, &schema, true)?;
@@ -108,6 +113,7 @@ impl DatafusionArrowPredicate {
108113
),
109114
rows_filtered,
110115
time,
116+
schema_mapping,
111117
})
112118
}
113119
}
@@ -123,6 +129,8 @@ impl ArrowPredicate for DatafusionArrowPredicate {
123129
false => batch.project(&self.projection)?,
124130
};
125131

132+
let batch = self.schema_mapping.map_partial_batch(batch)?;
133+
126134
// scoped timer updates on drop
127135
let mut timer = self.time.timer();
128136
match self
@@ -323,6 +331,7 @@ pub fn build_row_filter(
323331
metadata: &ParquetMetaData,
324332
reorder_predicates: bool,
325333
file_metrics: &ParquetFileMetrics,
334+
schema_mapping: Arc<dyn SchemaMapper>,
326335
) -> Result<Option<RowFilter>> {
327336
let rows_filtered = &file_metrics.pushdown_rows_filtered;
328337
let time = &file_metrics.pushdown_eval_time;
@@ -360,6 +369,7 @@ pub fn build_row_filter(
360369
metadata,
361370
rows_filtered.clone(),
362371
time.clone(),
372+
Arc::clone(&schema_mapping),
363373
)?;
364374

365375
filters.push(Box::new(filter));
@@ -372,6 +382,7 @@ pub fn build_row_filter(
372382
metadata,
373383
rows_filtered.clone(),
374384
time.clone(),
385+
Arc::clone(&schema_mapping),
375386
)?;
376387

377388
filters.push(Box::new(filter));
@@ -387,6 +398,7 @@ pub fn build_row_filter(
387398
metadata,
388399
rows_filtered.clone(),
389400
time.clone(),
401+
Arc::clone(&schema_mapping),
390402
)?;
391403

392404
filters.push(Box::new(filter));
@@ -398,15 +410,23 @@ pub fn build_row_filter(
398410

399411
#[cfg(test)]
400412
mod test {
401-
use super::*;
402413
use arrow::datatypes::Field;
414+
use arrow_schema::TimeUnit::Nanosecond;
415+
use parquet::arrow::arrow_reader::ParquetRecordBatchReaderBuilder;
416+
use parquet::arrow::parquet_to_arrow_schema;
417+
use parquet::file::reader::{FileReader, SerializedFileReader};
418+
use rand::prelude::*;
419+
420+
use crate::datasource::schema_adapter::DefaultSchemaAdapterFactory;
421+
use crate::datasource::schema_adapter::SchemaAdapterFactory;
422+
403423
use datafusion_common::ToDFSchema;
404424
use datafusion_expr::execution_props::ExecutionProps;
405425
use datafusion_expr::{cast, col, lit, Expr};
406426
use datafusion_physical_expr::create_physical_expr;
407-
use parquet::arrow::parquet_to_arrow_schema;
408-
use parquet::file::reader::{FileReader, SerializedFileReader};
409-
use rand::prelude::*;
427+
use datafusion_physical_plan::metrics::{Count, Time};
428+
429+
use super::*;
410430

411431
// We should ignore predicate that read non-primitive columns
412432
#[test]
@@ -473,6 +493,86 @@ mod test {
473493
);
474494
}
475495

496+
#[test]
497+
fn test_filter_type_coercion() {
498+
let testdata = crate::test_util::parquet_test_data();
499+
let file = std::fs::File::open(format!("{testdata}/alltypes_plain.parquet"))
500+
.expect("opening file");
501+
502+
let parquet_reader_builder =
503+
ParquetRecordBatchReaderBuilder::try_new(file).expect("creating reader");
504+
let metadata = parquet_reader_builder.metadata().clone();
505+
let file_schema = parquet_reader_builder.schema().clone();
506+
507+
// This is the schema we would like to coerce to,
508+
// which is different from the physical schema of the file.
509+
let table_schema = Schema::new(vec![Field::new(
510+
"timestamp_col",
511+
DataType::Timestamp(Nanosecond, Some(Arc::from("UTC"))),
512+
false,
513+
)]);
514+
515+
let schema_adapter =
516+
DefaultSchemaAdapterFactory {}.create(Arc::new(table_schema.clone()));
517+
let (schema_mapping, _) = schema_adapter
518+
.map_schema(&file_schema)
519+
.expect("creating schema mapping");
520+
521+
let mut parquet_reader = parquet_reader_builder.build().expect("building reader");
522+
523+
// Parquet file is small, we only need 1 recordbatch
524+
let first_rb = parquet_reader
525+
.next()
526+
.expect("expected record batch")
527+
.expect("expected error free record batch");
528+
529+
// Test all should fail
530+
let expr = col("timestamp_col").lt(Expr::Literal(
531+
ScalarValue::TimestampNanosecond(Some(1), Some(Arc::from("UTC"))),
532+
));
533+
let expr = logical2physical(&expr, &table_schema);
534+
let candidate = FilterCandidateBuilder::new(expr, &file_schema, &table_schema)
535+
.build(&metadata)
536+
.expect("building candidate")
537+
.expect("candidate expected");
538+
539+
let mut row_filter = DatafusionArrowPredicate::try_new(
540+
candidate,
541+
&file_schema,
542+
&metadata,
543+
Count::new(),
544+
Time::new(),
545+
Arc::clone(&schema_mapping),
546+
)
547+
.expect("creating filter predicate");
548+
549+
let filtered = row_filter.evaluate(first_rb.clone());
550+
assert!(matches!(filtered, Ok(a) if a == BooleanArray::from(vec![false; 8])));
551+
552+
// Test all should pass
553+
let expr = col("timestamp_col").gt(Expr::Literal(
554+
ScalarValue::TimestampNanosecond(Some(0), Some(Arc::from("UTC"))),
555+
));
556+
let expr = logical2physical(&expr, &table_schema);
557+
let candidate = FilterCandidateBuilder::new(expr, &file_schema, &table_schema)
558+
.build(&metadata)
559+
.expect("building candidate")
560+
.expect("candidate expected");
561+
562+
let mut row_filter = DatafusionArrowPredicate::try_new(
563+
candidate,
564+
&file_schema,
565+
&metadata,
566+
Count::new(),
567+
Time::new(),
568+
schema_mapping,
569+
)
570+
.expect("creating filter predicate");
571+
572+
let filtered = row_filter.evaluate(first_rb);
573+
assert!(matches!(filtered, Ok(a) if a == BooleanArray::from(vec![true; 8])));
574+
}
575+
476576
#[test]
477577
fn test_remap_projection() {
478578
let mut rng = thread_rng();

datafusion/core/src/datasource/schema_adapter.rs

Lines changed: 45 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -75,9 +75,21 @@ pub trait SchemaAdapter: Send + Sync {
7575

7676
/// Creates a `SchemaMapping` that can be used to cast or map the columns
7777
/// from the file schema to the table schema.
78-
pub trait SchemaMapper: Send + Sync {
78+
pub trait SchemaMapper: Debug + Send + Sync {
7979
/// Adapts a `RecordBatch` to match the `table_schema` using the stored mapping and conversions.
8080
fn map_batch(&self, batch: RecordBatch) -> datafusion_common::Result<RecordBatch>;
81+
82+
/// Adapts a [`RecordBatch`] that does not have all the columns from the
83+
/// file schema.
84+
///
85+
/// This method is used when applying a filter to a subset of the columns during
86+
/// an `ArrowPredicate`.
87+
///
88+
/// This method is slower than `map_batch` as it looks up columns by name.
89+
fn map_partial_batch(
90+
&self,
91+
batch: RecordBatch,
92+
) -> datafusion_common::Result<RecordBatch>;
8193
}
8294

8395
#[derive(Clone, Debug, Default)]
@@ -185,6 +197,31 @@ impl SchemaMapper for SchemaMapping {
185197
let record_batch = RecordBatch::try_new_with_options(schema, cols, &options)?;
186198
Ok(record_batch)
187199
}
200+
201+
fn map_partial_batch(
202+
&self,
203+
batch: RecordBatch,
204+
) -> datafusion_common::Result<RecordBatch> {
205+
let batch_cols = batch.columns().to_vec();
206+
let schema = batch.schema();
207+
208+
let mut cols = vec![];
209+
let mut fields = vec![];
210+
for (i, f) in schema.fields().iter().enumerate() {
211+
let table_field = self.table_schema.field_with_name(f.name());
212+
if let Ok(tf) = table_field {
213+
cols.push(cast(&batch_cols[i], tf.data_type())?);
214+
fields.push(tf.clone());
215+
}
216+
}
217+
218+
// Necessary to handle empty batches
219+
let options = RecordBatchOptions::new().with_row_count(Some(batch.num_rows()));
220+
221+
let schema = Arc::new(Schema::new(fields));
222+
let record_batch = RecordBatch::try_new_with_options(schema, cols, &options)?;
223+
Ok(record_batch)
224+
}
188225
}
189226

190227
#[cfg(test)]
@@ -337,5 +374,12 @@ mod tests {
337374

338375
Ok(RecordBatch::try_new(schema, new_columns).unwrap())
339376
}
377+
378+
fn map_partial_batch(
379+
&self,
380+
batch: RecordBatch,
381+
) -> datafusion_common::Result<RecordBatch> {
382+
self.map_batch(batch)
383+
}
340384
}
341385
}

0 commit comments

Comments
 (0)