-
Notifications
You must be signed in to change notification settings - Fork 1.2k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
feat: Add bloom filter metric to ParquetExec #8772
feat: Add bloom filter metric to ParquetExec #8772
Conversation
datafusion/core/src/datasource/physical_plan/parquet/metrics.rs
Outdated
Show resolved
Hide resolved
datafusion/core/src/datasource/physical_plan/parquet/metrics.rs
Outdated
Show resolved
Hide resolved
I think the CI failure https://github.com/apache/arrow-datafusion/actions/runs/7430577648/job/20220434682?pr=8772 is unrelated to the changes in this PR (and was fixed on main -- so this branch should pass next time it is merged to main) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This looks great @my-vegetable-has-exploded -- thank you for the contribution
I think we should consider the different name suggested by @simonvandel but otherwise i think this PR is pretty much ready to go!
format!( | ||
", pruning_predicate={} [{}]", | ||
pre.predicate_expr(), | ||
pre.literal_guarantees() |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
this is a neat idea to display guarantees 👍
.as_str(), | ||
Some(0), | ||
Some(1), | ||
// we don't support pruning by statistics for in_list with more than 20 elements currently |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Also I think it the pruning by bloom filters happens first, so it may not even try to prune by statistics
👍
@@ -61,7 +61,7 @@ Filter: parquet_table.column1 != Int32(42) | |||
physical_plan | |||
CoalesceBatchesExec: target_batch_size=8192 | |||
--FilterExec: column1@0 != 42 | |||
----ParquetExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..101], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:101..202], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:202..303], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:303..403]]}, projection=[column1], predicate=column1@0 != 42, pruning_predicate=column1_min@0 != 42 OR 42 != column1_max@1 | |||
----ParquetExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..101], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:101..202], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:202..303], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:303..403]]}, projection=[column1], predicate=column1@0 != 42, pruning_predicate=column1_min@0 != 42 OR 42 != column1_max@1 [column1 not in (42)] |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
what do you think about adding a header to the guarantees to it was more explicit for interpreting?
----ParquetExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..101], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:101..202], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:202..303], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:303..403]]}, projection=[column1], predicate=column1@0 != 42, pruning_predicate=column1_min@0 != 42 OR 42 != column1_max@1 [column1 not in (42)] | |
----ParquetExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..101], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:101..202], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:202..303], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:303..403]]}, projection=[column1], predicate=column1@0 != 42, pruning_predicate=column1_min@0 != 42 OR 42 != column1_max@1 required_guarantees=[column1 not in (42)] |
b88b7ea
to
ecfccdd
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Looks great -- thank you very much @my-vegetable-has-exploded
Which issue does this PR close?
Closes #8767,
Closes #8768
Rationale for this change
What changes are included in this PR?
row_groups_pruned
metric into two parts, includingrow_groups_pruned_sbbf
androw_groups_pruned_sbbf
.row_group_pruning.rs
, by checking value ofrow_groups_pruned_sbbf
androw_groups_pruned_sbbf
LiteralGuarantees
ofParquetExec
Are these changes tested?
Are there any user-facing changes?