Skip to content
Open
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
208 changes: 208 additions & 0 deletions datafusion/core/tests/parquet/dynamic_row_group_pruning.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,208 @@
// 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.

//! End-to-end test for **runtime row-group pruning** driven by a TopK
//! `SortExec`'s `DynamicFilterPhysicalExpr`.
//!
//! A 5-row-group parquet file is constructed with disjoint statistics on
//! the sort column (`v`): row group `i` contains values
//! `[i*100, (i+1)*100)`. The query `ORDER BY v DESC LIMIT 5` fills the
//! TopK heap from the row group with the largest values; the threshold
//! then proves the remaining row groups cannot contribute. The runtime
//! `RowGroupPruner` in the parquet scan must observe the tightened
//! threshold and increment `row_groups_pruned_dynamic_filter`.
//!
//! We assert a property (`pruned >= 1`) rather than an exact count
//! because batch-arrival timing affects how soon the TopK heap fills,
//! and we don't want this test to become flaky.

use std::sync::Arc;

use arrow::array::{ArrayRef, Int64Array, RecordBatch};
use arrow_schema::{DataType, Field, Schema};

use crate::parquet::Unit::RowGroup;
use crate::parquet::{ContextWithParquet, Scenario};

/// Build five `RecordBatch`es whose `v` column ranges are disjoint:
/// batch `i` carries `v` values `[i*100, (i+1)*100)`. When written with
/// `max_row_group_row_count = 100` each batch lands in its own row group.
fn build_five_disjoint_batches(schema: &Arc<Schema>) -> Vec<RecordBatch> {
(0..5i64)
.map(|rg| {
let base = rg * 100;
let values: Vec<i64> = (base..base + 100).collect();
let col: ArrayRef = Arc::new(Int64Array::from(values));
RecordBatch::try_new(Arc::clone(schema), vec![col]).unwrap()
})
.collect()
}

/// Build five `RecordBatch`es in *descending* value order: batch 0 holds
/// `v ∈ [400, 500)`, batch 4 holds `v ∈ [0, 100)`. The physical row-group
/// order on disk therefore does **not** match the order a `ORDER BY v ASC`
/// query wants — sort-pushdown's `reorder_by_statistics` must rearrange
/// the access plan so the scan reads RG 4 first, then RG 3, etc.
fn build_five_disjoint_batches_desc(schema: &Arc<Schema>) -> Vec<RecordBatch> {
(0..5i64)
.map(|rg| {
let base = (4 - rg) * 100;
let values: Vec<i64> = (base..base + 100).collect();
let col: ArrayRef = Arc::new(Int64Array::from(values));
RecordBatch::try_new(Arc::clone(schema), vec![col]).unwrap()
})
.collect()
}

/// `ORDER BY v DESC LIMIT 5` against a 5-RG file with disjoint per-RG
/// stats must trigger runtime RG pruning: the first RG read fills the
/// heap, and the tightened threshold proves every other RG unreachable.
#[tokio::test]
async fn dynamic_rg_pruning_metric_fires_for_topk_descending_limit() {
let schema = Arc::new(Schema::new(vec![Field::new("v", DataType::Int64, false)]));
let batches = build_five_disjoint_batches(&schema);

// `with_custom_data` honors the custom schema + batches and ignores
// `Scenario`. `Unit::RowGroup(100)` enables `pushdown_filters`, which
// is required for the TopK dynamic filter to reach the parquet scan.
let mut ctx = ContextWithParquet::with_custom_data(
Scenario::Int,
RowGroup(100),
Arc::clone(&schema),
batches,
)
.await;

let output = ctx.query("SELECT v FROM t ORDER BY v DESC LIMIT 5").await;

assert_eq!(output.result_rows, 5, "query must return LIMIT rows",);

let pruned = output
.row_groups_pruned_dynamic_filter()
.expect("`row_groups_pruned_dynamic_filter` metric must be registered");
assert!(
pruned >= 1,
"dynamic RG pruner must skip at least one row group; \
pruned={pruned}\n{}",
output.description(),
);
}

/// Regression for the rg_plan / `reorder_by_statistics` ordering bug.
///
/// When `sort_order_for_reorder` is set on the parquet scan,
/// `prepare_access_plan` calls
/// [`PreparedAccessPlan::reorder_by_statistics`], which rearranges
/// `row_group_indexes` so the decoder reads row groups in stats-optimal
/// order (smallest-min first for ASC, etc.). The stream's per-RG plan
/// (`rg_plan`) — which the runtime pruner walks one entry at a time —
/// **must use this reordered list**, not the access plan's natural
/// (index-ascending) order. Otherwise the pruner would consult the
/// metadata of RG K while the decoder is actually about to yield RG K',
/// silently producing wrong results.
///
/// This test makes the failure visible:
///
/// - File is written with RGs in *descending* `v` order (RG 0 has the
/// largest values, RG 4 has the smallest).
/// - Query is `ORDER BY v ASC LIMIT 5`, so sort-pushdown reorders the
/// access plan to read RG 4 first, then RG 3, etc.
/// - The smallest five values (which form the entire correct LIMIT
/// answer) live in RG 4 alone. After they are emitted, the TopK
/// threshold tightens enough that the per-RG pruner skips every other
/// RG.
///
/// Without the fix, `rg_plan` would be `[0, 1, 2, 3, 4]` while the
/// decoder reads `[4, 3, 2, 1, 0]`. The first yielded reader (for RG 4
/// in the decoder) would be tracked as if it were RG 0, the pruner
/// would check RG 1's stats (id range 300..400) against a threshold
/// already tightened to `v < 5`, prune RG 1 (because nothing in
/// 300..400 can satisfy `v < 5`), and then the rebuild via
/// `into_builder` would scan a row group whose data does not match its
/// expected metadata. The query would return fewer than five rows or
/// the wrong rows.
#[tokio::test]
async fn dynamic_rg_pruning_handles_sort_pushdown_reorder() {
let schema = Arc::new(Schema::new(vec![Field::new("v", DataType::Int64, false)]));
let batches = build_five_disjoint_batches_desc(&schema);

let mut ctx = ContextWithParquet::with_custom_data(
Scenario::Int,
RowGroup(100),
Arc::clone(&schema),
batches,
)
.await;

let output = ctx.query("SELECT v FROM t ORDER BY v ASC LIMIT 5").await;

// Correctness — the five smallest values in the file are 0..=4.
// If `rg_plan` is misaligned with the decoder's read order, the
// pruner consults the wrong RG's stats and the result row count or
// values would drift.
assert_eq!(output.result_rows, 5, "query must return LIMIT rows");
let formatted = output.pretty_results();
for v in 0..=4i64 {
assert!(
formatted.contains(&format!("| {v} ")),
"output must contain the smallest value {v}; got:\n{formatted}",
);
}

// Behavior — the per-RG pruner must engage. We don't pin the exact
// count (batch-arrival timing affects how soon the heap fills); we
// only require that at least one row group is skipped at runtime.
let pruned = output
.row_groups_pruned_dynamic_filter()
.expect("`row_groups_pruned_dynamic_filter` metric must be registered");
assert!(
pruned >= 1,
"with `sort_order_for_reorder` active and a tight TopK, the \
runtime pruner must skip at least one row group; pruned={pruned}\n{}",
output.description(),
);
}

/// A query without ORDER BY does not produce a TopK and therefore no
/// `DynamicFilterPhysicalExpr` reaches the scan. The runtime pruner must
/// stay quiet — the metric should be 0.
#[tokio::test]
async fn dynamic_rg_pruning_metric_quiet_without_topk() {
let schema = Arc::new(Schema::new(vec![Field::new("v", DataType::Int64, false)]));
let batches = build_five_disjoint_batches(&schema);

let mut ctx = ContextWithParquet::with_custom_data(
Scenario::Int,
RowGroup(100),
Arc::clone(&schema),
batches,
)
.await;

// Plain `SELECT *` — no sort, no limit, no dynamic filter.
let output = ctx.query("SELECT v FROM t").await;
assert_eq!(output.result_rows, 500);

let pruned = output.row_groups_pruned_dynamic_filter().unwrap_or(0);
assert_eq!(
pruned,
0,
"without TopK there is no dynamic filter, so the runtime pruner \
must not fire; pruned={pruned}\n{}",
output.description(),
);
}
14 changes: 14 additions & 0 deletions datafusion/core/tests/parquet/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,7 @@ use tempfile::NamedTempFile;

mod content_defined_chunking;
mod custom_reader;
mod dynamic_row_group_pruning;
#[cfg(feature = "parquet_encryption")]
mod encryption;
mod expr_adapter;
Expand Down Expand Up @@ -147,6 +148,12 @@ struct TestOutput {
}

impl TestOutput {
/// Pretty-printed result batches, useful for asserting concrete row
/// values in regression tests.
fn pretty_results(&self) -> &str {
&self.pretty_results
}

/// retrieve the value of the named metric, if any
fn metric_value(&self, metric_name: &str) -> Option<usize> {
if let Some(pm) = self.pruning_metric(metric_name) {
Expand Down Expand Up @@ -259,6 +266,13 @@ impl TestOutput {
.map(|pm| pm.total_pruned())
}

/// The number of row groups pruned at runtime by the dynamic
/// row-group pruner (e.g. driven by a TopK `SortExec` threshold
/// pushed down via `DynamicFilterPhysicalExpr`).
fn row_groups_pruned_dynamic_filter(&self) -> Option<usize> {
self.metric_value("row_groups_pruned_dynamic_filter")
}

fn description(&self) -> String {
format!(
"Input:\n{}\nQuery:\n{}\nOutput:\n{}\nMetrics:\n{}",
Expand Down
72 changes: 0 additions & 72 deletions datafusion/datasource-parquet/src/access_plan.rs
Original file line number Diff line number Diff line change
Expand Up @@ -115,24 +115,6 @@ pub enum RowGroupAccess {
Selection(RowSelection),
}

/// A consecutive set of row groups that share the same row filter requirement.
#[derive(Debug, Clone, PartialEq)]
pub(crate) struct RowGroupRun {
/// True if this run needs row filter evaluation.
pub(crate) needs_filter: bool,
/// The access plan for this run.
pub(crate) access_plan: ParquetAccessPlan,
}

impl RowGroupRun {
fn new(needs_filter: bool, access_plan: ParquetAccessPlan) -> Self {
Self {
needs_filter,
access_plan,
}
}
}

impl RowGroupAccess {
/// Return true if this row group should be scanned
pub fn should_scan(&self) -> bool {
Expand Down Expand Up @@ -213,12 +195,6 @@ impl ParquetAccessPlan {
&self.fully_matched
}

/// Return true if any scanned row group is fully matched.
fn has_fully_matched(&self) -> bool {
self.row_group_index_iter()
.any(|idx| self.is_fully_matched(idx))
}

/// Set to scan only the [`RowSelection`] in the specified row group.
///
/// Behavior is different depending on the existing access
Expand Down Expand Up @@ -404,54 +380,6 @@ impl ParquetAccessPlan {
self.row_groups
}

/// Split this plan into consecutive row group runs that share the same row
/// filter requirement.
pub(crate) fn split_runs(self, needs_filter: bool) -> Vec<RowGroupRun> {
if !needs_filter || !self.has_fully_matched() {
return vec![RowGroupRun::new(needs_filter, self)];
}

let num_row_groups = self.row_groups.len();
let row_groups = self.row_groups;
let fully_matched = self.fully_matched;
let mut runs: Vec<RowGroupRun> = Vec::new();

for (idx, (access, fully_matched)) in
row_groups.into_iter().zip(fully_matched).enumerate()
{
if !access.should_scan() {
continue;
}

let row_group_needs_filter = !fully_matched;
if let Some(run) = runs
.last_mut()
.filter(|run| run.needs_filter == row_group_needs_filter)
{
run.access_plan.set(idx, access);
if fully_matched {
run.access_plan.mark_fully_matched(idx);
}
} else {
let mut run_plan = ParquetAccessPlan::new_none(num_row_groups);
run_plan.set(idx, access);
if fully_matched {
run_plan.mark_fully_matched(idx);
}
runs.push(RowGroupRun::new(row_group_needs_filter, run_plan));
}
}

if runs.is_empty() {
vec![RowGroupRun::new(
needs_filter,
ParquetAccessPlan::new_none(num_row_groups),
)]
} else {
runs
}
}

/// Prepare this plan and resolve to the final `PreparedAccessPlan`
pub(crate) fn prepare(
self,
Expand Down
14 changes: 14 additions & 0 deletions datafusion/datasource-parquet/src/metrics.rs
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,14 @@ pub struct ParquetFileMetrics {
pub limit_pruned_row_groups: PruningMetrics,
/// Number of row groups pruned by statistics
pub row_groups_pruned_statistics: PruningMetrics,
/// Number of row groups pruned at runtime by a dynamic predicate
/// (e.g. the threshold expression a TopK `SortExec` pushes down).
///
/// Unlike [`Self::row_groups_pruned_statistics`], which is decided once
/// at access-plan time, this counter reflects row groups that survived
/// the initial pruning but were proved unreachable mid-scan after the
/// dynamic filter tightened.
pub row_groups_pruned_dynamic_filter: Count,
/// Total number of bytes scanned
pub bytes_scanned: Count,
/// Total rows filtered out by predicates pushed into parquet scan
Expand Down Expand Up @@ -198,6 +206,11 @@ impl ParquetFileMetrics {
.with_category(MetricCategory::Rows)
.gauge("predicate_cache_records", partition);

let row_groups_pruned_dynamic_filter = MetricBuilder::new(metrics)
.with_new_label("filename", filename.to_string())
.with_type(MetricType::Summary)
.counter("row_groups_pruned_dynamic_filter", partition);

Self {
files_ranges_pruned_statistics,
predicate_evaluation_errors,
Expand All @@ -217,6 +230,7 @@ impl ParquetFileMetrics {
scan_efficiency_ratio,
predicate_cache_inner_records,
predicate_cache_records,
row_groups_pruned_dynamic_filter,
}
}

Expand Down
Loading
Loading