Skip to content
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

fix guarantees in allways_true of PruningPredicate #8732

Merged
merged 4 commits into from
Jan 3, 2024

Conversation

my-vegetable-has-exploded
Copy link
Contributor

@my-vegetable-has-exploded my-vegetable-has-exploded commented Jan 3, 2024

Which issue does this PR close?

Closes #8685

Rationale for this change

If predicte_expr get unhandled, it will return true in always_true.

https://github.com/apache/arrow-datafusion/blob/9a6cc889a40e4740bfc859557a9ca9c8d043891e/datafusion/core/src/physical_optimizer/pruning.rs#L914

https://github.com/apache/arrow-datafusion/blob/9a6cc889a40e4740bfc859557a9ca9c8d043891e/datafusion/core/src/physical_optimizer/pruning.rs#L299-L302

Then this PruningPredicate will be filtered here.

https://github.com/apache/arrow-datafusion/blob/9a6cc889a40e4740bfc859557a9ca9c8d043891e/datafusion/core/src/datasource/physical_plan/parquet/mod.rs#L122-L135

If an expr can't be handled as predicate_expr(like in_list with more than 20 elements) but hava some literalguarantees , the PruningPredicate may also be filtered.

So , we need to check PruningPredicate in allways_true function.

What changes are included in this PR?

Are these changes tested?

Are there any user-facing changes?

@github-actions github-actions bot added the core Core DataFusion crate label Jan 3, 2024
@my-vegetable-has-exploded my-vegetable-has-exploded changed the title Minor: fix check guarantees in allways_true of PruningPredicate Minor: fix guarantees in allways_true of PruningPredicate Jan 3, 2024
@my-vegetable-has-exploded
Copy link
Contributor Author

my-vegetable-has-exploded commented Jan 3, 2024

May it is better to add some integration test? But I don't know how check whether bloomfilter works with current code. I check it by the metric I add myself(like https://github.com/apache/arrow-datafusion/compare/main...my-vegetable-has-exploded:arrow-datafusion:metric-sbbf?expand=1) , Should I add it to this branch?

@alamb
Copy link
Contributor

alamb commented Jan 3, 2024

Thank you @my-vegetable-has-exploded -- I will review this later today.

This also seems somewhat related to #7869 which I plan to work on this week. It is another case of not pruning even when there is sufficient information to do so

@alamb alamb changed the title Minor: fix guarantees in allways_true of PruningPredicate fix guarantees in allways_true of PruningPredicate Jan 3, 2024
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 again @my-vegetable-has-exploded and @domyway -- I reviewed the code carefully and I have pushed a test to this branch

While reviewing the initial PR that added this functionalty #4280 (which I added it seems 🤔 )

The idea seems to be to entirely skipping statistics for pruning predicates that
will never prune anything out, which seems reasonable, but when I
looked at the code it seems like we always will decode the parquet
statistics anyways, and the pruning predicate only creates predicates on demand.
Thus, I don't think skipping the entire pruning predicate saves much work at all.

I'll double check in a follow on PR, but I think we might just be able to remove that code entirely.

@alamb
Copy link
Contributor

alamb commented Jan 3, 2024

May it is better to add some integration test? But I don't know how check whether bloomfilter works with current code. I check it by the metric I add myself(like https://github.com/apache/arrow-datafusion/compare/main...my-vegetable-has-exploded:arrow-datafusion:metric-sbbf?expand=1) , Should I add it to this branch?

I think an integration test as well as the bloom filter metrics would be good.

Here are my recommended follow on steps

  1. One PR to add the new metrics to distinguish filtering on bloom filters vs statistics
  2. One PR with some integration tests to verify bloom filters are actually pruning (maybe following how it is done in https://github.com/apache/arrow-datafusion/blob/1179a76567892b259c88f08243ee01f05c4c3d5c/datafusion/core/tests/parquet/row_group_pruning.rs#L42)

@alamb alamb merged commit ad4b7b7 into apache:main Jan 3, 2024
22 checks passed
@my-vegetable-has-exploded
Copy link
Contributor Author

my-vegetable-has-exploded commented Jan 4, 2024

I think an integration test as well as the bloom filter metrics would be good.

Thanks @alamb, I will handle it later.

@alamb
Copy link
Contributor

alamb commented Jan 5, 2024

filed #8767 and #8768 to track this

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
core Core DataFusion crate
Projects
None yet
Development

Successfully merging this pull request may close these issues.

Regression: bloom filters are not being used in Parquet queries
2 participants