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

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

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


##########
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:
   If we allow min/max values to be lower/upper bounds and not part of the real 
data (as @wgtmac suggested) then there are only two scenarios we can be sure 
the requested value is in the row group:
   * Searching for `null` and the number of nulls (is specified and) is greater 
than `0`
   * The min value and the max value are both equal to the value we are 
searching for
   If we expect min/max values to be part of the row groups then we can extend 
the related statements but it might not be a safe choice.
   
   Dictionary filter is much more straightforward. Bloom filter should be used 
only if the requested value is not in the dictionary (or there is no dictionary 
at all) and there are pages that are not dictionary encoded. So your change 
clearly makes sense to differenciate the three potential results and move 
forward to bloom filters only in case of `BLOCK_MIGHT_MATCH`.
   
   





> 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