Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
68 changes: 67 additions & 1 deletion datafusion/core/src/datasource/physical_plan/parquet.rs
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,7 @@ mod tests {
use crate::test::object_store::local_unpartitioned_file;
use arrow::array::{
ArrayRef, AsArray, Date64Array, Int32Array, Int64Array, Int8Array, StringArray,
StringViewArray, StructArray,
StringViewArray, StructArray, TimestampNanosecondArray,
};
use arrow::datatypes::{DataType, Field, Fields, Schema, SchemaBuilder};
use arrow::record_batch::RecordBatch;
Expand Down Expand Up @@ -960,6 +960,72 @@ mod tests {
assert_eq!(read, 2, "Expected 2 rows to match the predicate");
}

#[tokio::test]
async fn evolved_schema_column_type_filter_timestamp_units() {
// The table and filter have a common data type
// The table schema is in milliseconds, but the file schema is in nanoseconds
let c1: ArrayRef = Arc::new(TimestampNanosecondArray::from(vec![
Some(1_000_000_000), // 1970-01-01T00:00:01Z
Some(2_000_000_000), // 1970-01-01T00:00:02Z
Some(3_000_000_000), // 1970-01-01T00:00:03Z
Some(4_000_000_000), // 1970-01-01T00:00:04Z
]));
let batch = create_batch(vec![("c1", c1.clone())]);
let table_schema = Arc::new(Schema::new(vec![Field::new(
"c1",
DataType::Timestamp(TimeUnit::Millisecond, Some("UTC".into())),
false,
)]));
// One row should match, 2 pruned via page index, 1 pruned via filter pushdown
let filter = col("c1").eq(lit(ScalarValue::TimestampMillisecond(
Some(1_000),
Some("UTC".into()),
)));
let rt = RoundTrip::new()
.with_predicate(filter)
.with_pushdown_predicate()
.with_page_index_predicate() // produces pages with 2 rows each (2 pages total for our data)
.with_table_schema(table_schema.clone())
.round_trip(vec![batch.clone()])
.await;
// There should be no predicate evaluation errors and we keep 1 row
let metrics = rt.parquet_exec.metrics().unwrap();
assert_eq!(get_value(&metrics, "predicate_evaluation_errors"), 0);
let read = rt
.batches
.unwrap()
.iter()
.map(|b| b.num_rows())
.sum::<usize>();
assert_eq!(read, 1, "Expected 1 rows to match the predicate");
assert_eq!(get_value(&metrics, "row_groups_pruned_statistics"), 0);
assert_eq!(get_value(&metrics, "page_index_rows_pruned"), 2);
assert_eq!(get_value(&metrics, "pushdown_rows_pruned"), 1);
// If we filter with a value that is completely out of the range of the data
// we prune at the row group level.
let filter = col("c1").eq(lit(ScalarValue::TimestampMillisecond(
Some(5_000),
Some("UTC".into()),
)));
let rt = RoundTrip::new()
.with_predicate(filter)
.with_pushdown_predicate()
.with_table_schema(table_schema)
.round_trip(vec![batch])
.await;
// There should be no predicate evaluation errors and we keep 0 rows
let metrics = rt.parquet_exec.metrics().unwrap();
assert_eq!(get_value(&metrics, "predicate_evaluation_errors"), 0);
let read = rt
.batches
.unwrap()
.iter()
.map(|b| b.num_rows())
.sum::<usize>();
assert_eq!(read, 0, "Expected 0 rows to match the predicate");
assert_eq!(get_value(&metrics, "row_groups_pruned_statistics"), 1);
}

#[tokio::test]
async fn evolved_schema_disjoint_schema_filter() {
let c1: ArrayRef =
Expand Down
136 changes: 131 additions & 5 deletions datafusion/datasource-parquet/src/opener.rs
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@ use datafusion_common::pruning::{
};
use datafusion_common::{exec_err, Result};
use datafusion_datasource::PartitionedFile;
use datafusion_physical_expr::PhysicalExprSchemaRewriter;
use datafusion_physical_expr_common::physical_expr::PhysicalExpr;
use datafusion_physical_optimizer::pruning::PruningPredicate;
use datafusion_physical_plan::metrics::{Count, ExecutionPlanMetricsSet, MetricBuilder};
Expand Down Expand Up @@ -117,7 +118,6 @@ impl FileOpener for ParquetOpener {

let projected_schema =
SchemaRef::from(self.logical_file_schema.project(&self.projection)?);
let schema_adapter_factory = Arc::clone(&self.schema_adapter_factory);
let schema_adapter = self
.schema_adapter_factory
.create(projected_schema, Arc::clone(&self.logical_file_schema));
Expand Down Expand Up @@ -159,7 +159,7 @@ impl FileOpener for ParquetOpener {
if let Some(pruning_predicate) = pruning_predicate {
// The partition column schema is the schema of the table - the schema of the file
let mut pruning = Box::new(PartitionPruningStatistics::try_new(
vec![file.partition_values],
vec![file.partition_values.clone()],
partition_fields.clone(),
)?)
as Box<dyn PruningStatistics>;
Expand Down Expand Up @@ -248,10 +248,27 @@ impl FileOpener for ParquetOpener {
}
}

// Adapt the predicate to the physical file schema.
// This evaluates missing columns and inserts any necessary casts.
let predicate = predicate
.map(|p| {
PhysicalExprSchemaRewriter::new(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

😍

&physical_file_schema,
&logical_file_schema,
)
.with_partition_columns(
partition_fields.to_vec(),
file.partition_values,
)
.rewrite(p)
.map_err(ArrowError::from)
})
.transpose()?;

// Build predicates for this specific file
let (pruning_predicate, page_pruning_predicate) = build_pruning_predicates(
predicate.as_ref(),
&logical_file_schema,
&physical_file_schema,
&predicate_creation_errors,
);

Expand Down Expand Up @@ -288,11 +305,9 @@ impl FileOpener for ParquetOpener {
let row_filter = row_filter::build_row_filter(
&predicate,
&physical_file_schema,
&logical_file_schema,
builder.metadata(),
reorder_predicates,
&file_metrics,
&schema_adapter_factory,
);

match row_filter {
Expand Down Expand Up @@ -879,4 +894,115 @@ mod test {
assert_eq!(num_batches, 0);
assert_eq!(num_rows, 0);
}

#[tokio::test]
async fn test_prune_on_partition_value_and_data_value() {
let store = Arc::new(InMemory::new()) as Arc<dyn ObjectStore>;

// Note: number 3 is missing!
let batch = record_batch!(("a", Int32, vec![Some(1), Some(2), Some(4)])).unwrap();
let data_size =
write_parquet(Arc::clone(&store), "part=1/file.parquet", batch.clone()).await;

let file_schema = batch.schema();
let mut file = PartitionedFile::new(
"part=1/file.parquet".to_string(),
u64::try_from(data_size).unwrap(),
);
file.partition_values = vec![ScalarValue::Int32(Some(1))];

let table_schema = Arc::new(Schema::new(vec![
Field::new("part", DataType::Int32, false),
Field::new("a", DataType::Int32, false),
]));

let make_opener = |predicate| {
ParquetOpener {
partition_index: 0,
projection: Arc::new([0]),
batch_size: 1024,
limit: None,
predicate: Some(predicate),
logical_file_schema: file_schema.clone(),
metadata_size_hint: None,
metrics: ExecutionPlanMetricsSet::new(),
parquet_file_reader_factory: Arc::new(
DefaultParquetFileReaderFactory::new(Arc::clone(&store)),
),
partition_fields: vec![Arc::new(Field::new(
"part",
DataType::Int32,
false,
))],
pushdown_filters: true, // note that this is true!
reorder_filters: true,
enable_page_index: false,
enable_bloom_filter: false,
schema_adapter_factory: Arc::new(DefaultSchemaAdapterFactory),
enable_row_group_stats_pruning: false, // note that this is false!
coerce_int96: None,
}
};

let make_meta = || FileMeta {
object_meta: ObjectMeta {
location: Path::from("part=1/file.parquet"),
last_modified: Utc::now(),
size: u64::try_from(data_size).unwrap(),
e_tag: None,
version: None,
},
range: None,
extensions: None,
metadata_size_hint: None,
};

// Filter should match the partition value and data value
let expr = col("part").eq(lit(1)).or(col("a").eq(lit(1)));
let predicate = logical2physical(&expr, &table_schema);
let opener = make_opener(predicate);
let stream = opener
.open(make_meta(), file.clone())
.unwrap()
.await
.unwrap();
let (num_batches, num_rows) = count_batches_and_rows(stream).await;
assert_eq!(num_batches, 1);
assert_eq!(num_rows, 3);

// Filter should match the partition value but not the data value
let expr = col("part").eq(lit(1)).or(col("a").eq(lit(3)));
let predicate = logical2physical(&expr, &table_schema);
let opener = make_opener(predicate);
let stream = opener
.open(make_meta(), file.clone())
.unwrap()
.await
.unwrap();
let (num_batches, num_rows) = count_batches_and_rows(stream).await;
assert_eq!(num_batches, 1);
assert_eq!(num_rows, 3);

// Filter should not match the partition value but match the data value
let expr = col("part").eq(lit(2)).or(col("a").eq(lit(1)));
let predicate = logical2physical(&expr, &table_schema);
let opener = make_opener(predicate);
let stream = opener
.open(make_meta(), file.clone())
.unwrap()
.await
.unwrap();
let (num_batches, num_rows) = count_batches_and_rows(stream).await;
assert_eq!(num_batches, 1);
assert_eq!(num_rows, 1);
Copy link
Contributor Author

@adriangb adriangb Jun 19, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This assertion fails on main: all 3 rows are passed because the row filter cannot handle the partition columns. This PR somewhat coincidentally happens to allow the row filter to handle predicates that depend on partition and data columns!


// Filter should not match the partition value or the data value
let expr = col("part").eq(lit(2)).or(col("a").eq(lit(3)));
let predicate = logical2physical(&expr, &table_schema);
let opener = make_opener(predicate);
let stream = opener.open(make_meta(), file).unwrap().await.unwrap();
let (num_batches, num_rows) = count_batches_and_rows(stream).await;
assert_eq!(num_batches, 0);
assert_eq!(num_rows, 0);
}
}
Loading