[ 
https://issues.apache.org/jira/browse/PARQUET-2237?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17692124#comment-17692124
 ] 

ASF GitHub Bot commented on PARQUET-2237:
-----------------------------------------

gszadovszky commented on code in PR #1023:
URL: https://github.com/apache/parquet-mr/pull/1023#discussion_r1114171229


##########
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:
   Godd catch indeed, @yabola! Could you open a separate jira and maybe a PR 
for this finding?
   
   @wgtmac, performance. Let's see the following scenario. We have dictionary 
encoding but not for all the pages. We also have Bloom filter. Does it worth 
reading the dictionary to check if a value is in there knowing if it doesn't we 
still want to check the Bloom filter? I don't know the answer, maybe yes. But 
if it is a no, then the whole concept of this PR is questionable.
   For the case of all the pages are dictionary encoded we should not have 
Bloom filters therefore it doesn't really matter if we return 
`BLOCK_MIGHT_MATCH` or `BLOCK_MUST_MATCH` in case we find the interested values 
in the dictionary.
   Since we might already written some Bloom filters for fully dictionary 
encoded column chunks we should handle this scenario. But we can do it easily 
buy skipping reading Bloom filters in this case completely.





> 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)

Reply via email to