Skip to content

Conversation

@2010YOUY01
Copy link
Contributor

Which issue does this PR close?

Rationale for this change

See writeup in #18297
This PR is for the remaining metrics in DataSourceExec with parquet data source.

Demo

In datafusion-cli

CREATE EXTERNAL TABLE IF NOT EXISTS lineitem
STORED AS parquet
LOCATION '/Users/yongting/Code/datafusion/benchmarks/data/tpch_sf1/lineitem';

set datafusion.explain.analyze_level = summary;

explain analyze select *
from lineitem
where l_orderkey = 3000000;

+-------------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
| plan_type         | plan                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                 |
+-------------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
| Plan with Metrics | CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=5, elapsed_compute=48.677µs, output_bytes=1092.0 B]                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                |
|                   |   FilterExec: l_orderkey@0 = 3000000, metrics=[output_rows=5, elapsed_compute=1.65872ms, output_bytes=530.8 KB]                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                      |
|                   |     DataSourceExec: file_groups={14 groups: [[Users/yongting/Code/datafusion/benchmarks/data/tpch_sf1/lineitem/part-0.parquet:0..11525426], [Users/yongting/Code/datafusion/benchmarks/data/tpch_sf1/lineitem/part-0.parquet:11525426..20311205, Users/yongting/Code/datafusion/benchmarks/data/tpch_sf1/lineitem/part-1.parquet:0..2739647], [Users/yongting/Code/datafusion/benchmarks/data/tpch_sf1/lineitem/part-1.parquet:2739647..14265073], [Users/yongting/Code/datafusion/benchmarks/data/tpch_sf1/lineitem/part-1.parquet:14265073..20193593, Users/yongting/Code/datafusion/benchmarks/data/tpch_sf1/lineitem/part-2.parquet:0..5596906], [Users/yongting/Code/datafusion/benchmarks/data/tpch_sf1/lineitem/part-2.parquet:5596906..17122332], ...]}, projection=[l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment], file_type=parquet, predicate=l_orderkey@0 = 3000000, pruning_predicate=l_orderkey_null_count@2 != row_count@3 AND l_orderkey_min@0 <= 3000000 AND 3000000 <= l_orderkey_max@1, required_guarantees=[l_orderkey in (3000000)], metrics=[output_rows=19813, elapsed_compute=14ns, output_bytes=5.7 MB, files_ranges_pruned_statistics=21 total → 3 matched, page_index_rows_pruned=748901 total → 19813 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, bytes_scanned=2147308, metadata_load_time=1.794289ms] |
|                   |                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                      |
+-------------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
1 row(s) fetched.
Elapsed 0.081 seconds.

What changes are included in this PR?

Update row_groups_pruned_statistics, row_groups_pruned_bloom_filter, page_index_rows_pruned with the new PruningMetrics metric type.

The functional changes in the pr are in datafusion/datasource-parquet/src/*, it's only a few of lines, most changes are fixing tests.

Are these changes tested?

UTs are updated for the new metrics

Are there any user-facing changes?

No

@github-actions github-actions bot added documentation Improvements or additions to documentation core Core DataFusion crate datasource Changes to the datasource crate physical-plan Changes to the physical-plan crate labels Oct 28, 2025
metrics.row_groups_pruned_bloom_filter.add(1);
metrics.row_groups_pruned_bloom_filter.add_pruned(1);
self.access_plan.skip(idx)
} else if !stats.column_sbbf.is_empty() {
Copy link
Contributor Author

Choose a reason for hiding this comment

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

Before row_groups_matched_bloom_filter is not incremented if bf is not available, I think matched means not able to prune in the current stage, and this range should be further checked, so this should also be incremented.

Copy link
Contributor

@alamb alamb left a comment

Choose a reason for hiding this comment

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

Thank you @2010YOUY01 -- this looks great to me

/// Metric `files_ranges_pruned_statistics` tracks both pruned and matched count,
/// for testing purpose, here it only aggregate the `pruned` count.
fn files_ranges_pruned_statistics(&self) -> Option<usize> {
fn pruning_metric(&self, metric_name: &str) -> Option<(usize, usize)> {
Copy link
Contributor

Choose a reason for hiding this comment

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

I think it would help to document here what the two usize return values were (total rows pruned and total rows matched)

let plan = format!("{}", arrow::util::pretty::pretty_format_batches(&batches)?);
println!("{plan}");
assert_contains!(&plan, "row_groups_pruned_statistics=1");
assert_contains!(&plan, "row_groups_pruned_statistics=2 total → 1 matched");
Copy link
Contributor

Choose a reason for hiding this comment

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

this is so much clearer ❤️

@2010YOUY01
Copy link
Contributor Author

2010YOUY01 commented Oct 29, 2025

There are many test fixes due to the row_groups_matched_bloom_filter's behavior change, I think the updated behavior is more reasonable. I have also verified the test changes are expected.

The old behavior is, if bloom filter is not available, the 'matched' count will be 0, this PR changed matched count to total remaining row group count to further scan, so the metrics will be displayed as follow:

(Parquet scanner will first check statistics, next bloom filters for row group pruning)

  • Case 1: both stat and bf have successfully pruned row groups:
    row_groups_pruned_statistics=10 total -> 7 matched, row_group_pruned_bloom_filter=7 total -> 3 matched
  • Case 2: stat has successfully pruned row groups, bloom filter is not available
    row_groups_pruned_statistics=10 total -> 7 matched, row_group_pruned_bloom_filter=7 total -> 7 matched

Perhaps if bf is not available, we can let it show row_groups_pruned_bloom_filter=FILTER UNAVAILABLE instead, this can be done as a follow up.

@alamb
Copy link
Contributor

alamb commented Oct 29, 2025

Perhaps if bf is not available, we can let it show row_groups_pruned_bloom_filter=FILTER UNAVAILABLE instead, this can be done as a follow up.

I suggest filing this as a ticket (a good first issue for example)

@alamb
Copy link
Contributor

alamb commented Oct 29, 2025

I would actually suggest not printing anything if bloom filters are not available (rather than UNAVAILABLE)

@2010YOUY01 2010YOUY01 added this pull request to the merge queue Oct 30, 2025
Merged via the queue into apache:main with commit 9e9eb94 Oct 30, 2025
33 checks passed
github-merge-queue bot pushed a commit that referenced this pull request Oct 31, 2025
## Which issue does this PR close?

<!--
We generally require a GitHub issue to be filed for all bug fixes and
enhancements and this helps us generate change logs for our releases.
You can link an issue to this PR using the GitHub syntax. For example
`Closes #123` indicates that this PR will close issue #123.
-->

- Closes #.

## Rationale for this change

<!--
Why are you proposing this change? If this is already explained clearly
in the issue then this section is not needed.
Explaining clearly why changes are proposed helps reviewers understand
your changes and offer better suggestions for fixes.
-->
It's better to let pruning metrics in parquet displayed in an order that
is the same as the actual pruning order:
```
metrics=...files_ranges_pruned_statistics=21 total → 3 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_rows_pruned=748901 total → 19813 matched...
```
Now it's ordered alphabetically.
See #18321 (comment)
for reproducing.

## What changes are included in this PR?

<!--
There is no need to duplicate the description in the issue here but it
is sometimes worth providing a summary of the individual changes in this
PR.
-->
Update the sort key API in `MetricValue`, to let the parquet pruning
metrics display in the expected order.

## Are these changes tested?
UT
<!--
We typically require tests for all PRs in order to:
1. Prevent the code from being accidentally broken by subsequent changes
2. Serve as another way to document the expected behavior of the code

If tests are not included in your PR, please explain why (for example,
are they covered by existing tests)?
-->

## Are there any user-facing changes?
No
<!--
If there are user-facing changes then we may require documentation to be
updated before approving the PR.
-->

<!--
If there are any breaking changes to public APIs, please add the `api
change` label.
-->
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

core Core DataFusion crate datasource Changes to the datasource crate documentation Improvements or additions to documentation physical-plan Changes to the physical-plan crate

Projects

None yet

Development

Successfully merging this pull request may close these issues.

For parquet row-group/page pruning, use PruningMetrics for nicer metrics display

2 participants