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

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

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


##########
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:
   @gszadovszky @wgtmac @zhongyujiang Thank you very much for working on it.  I 
have some thoughts.
   
   > We can improve (?) the case when not all the pages are dictionary encoded
   
   1. I can't make sure if it is suitable to load dictionary even if pages are 
not all decoded. (I may choose not to change this behavior)
   
   2. However considering the origin `BloomFilter` bug in parquet v1, we might 
have to do something to avoid using `BloomFilter`(even if pages are all 
encoded).  
   In the code implementation we may have to use some flag to mark if 
dictionary `DictionaryFilter#expandDictionary` successfully (method will throw 
`IOException` and we can't `expandDictionary` again in `BloomFilterImpl`).
   Or we could also use `BLOCK_MUST_MATCH` like this PR.
   
   > StatisticsFilter: Because of the lower/upper bound issue we cannot really 
improve this (except for the specific case when min=max)
   
   If we only use it when min=max, I think it might not really improve . 
   
   





> 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