[
https://issues.apache.org/jira/browse/PARQUET-2237?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17692086#comment-17692086
]
ASF GitHub Bot commented on PARQUET-2237:
-----------------------------------------
yabola commented on code in PR #1023:
URL: https://github.com/apache/parquet-mr/pull/1023#discussion_r1114087072
##########
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##########
@@ -289,8 +320,14 @@ public <T extends Comparable<T>> Boolean visit(Lt<T> lt) {
T value = lt.getValue();
- // drop if value <= min
- return stats.compareMinToValue(value) >= 0;
+ // we are looking for records where v < someValue
+ if (stats.compareMinToValue(value) >= 0) {
+ // drop if value <= min
+ return BLOCK_CANNOT_MATCH;
+ } else {
+ // if value > min, we must take it
+ return BLOCK_MUST_MATCH;
Review Comment:
@wgtmac @gszadovszky I think it might be bug here because we use
`PLAIN_DICTIONARY` in parquet v1?
Bloomfilter will avoid generating only when the dictionary types are all
`RLE_DICTIONARY`? ( I can fix this bug in another PR, I might have to check
more)
https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java#L884
I quick tested on it and found
`EncodingStats#hasNonDictionaryEncodedPages` return false (means all pages are
encoded) but `BloomFilterImpl` can still read `BloomFilter`.
> Improve performance when filters in RowGroupFilter can match exactly
> --------------------------------------------------------------------
>
> Key: PARQUET-2237
> URL: https://issues.apache.org/jira/browse/PARQUET-2237
> Project: Parquet
> Issue Type: Improvement
> Reporter: Mars
> Priority: Major
>
> If we can accurately judge by the minMax status, we don’t need to load the
> dictionary from filesystem and compare one by one anymore.
> Similarly , Bloomfilter needs to load from filesystem, it may costs time and
> memory. If we can exactly determine the existence/nonexistence of the value
> from minMax or dictionary filters , then we can avoid using Bloomfilter to
> Improve performance.
> For example,
> # read data greater than {{x1}} in the block, if minMax in status is all
> greater than {{{}x1{}}}, then we don't need to read dictionary and compare
> one by one.
> # If we already have page dictionaries and have compared one by one, we
> don't need to read BloomFilter and compare.
--
This message was sent by Atlassian Jira
(v8.20.10#820010)