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

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

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


##########
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:
   @yabola, I think you misunderstand how dictionary/filtering works. The 
dictionary contains all of the values which the dictionary encoded pages may 
contain. These pages actually not contain the values but the indices 
referencing to the related values in the dictionary.
   
   So, if a searched element can be found in the dictionary you may return 
`BLOCK_MUST_MATCH` even if only one page is dictionary encoded. For example if 
the filter is `x > 1` then any element in the dictionary `> 1` would fulfill 
the filter so the `BLOCK_MUST_MATCH`.
   If the dictionary does not contain any of the searched elements (for the 
previous example every elements are `<= 1`) then you may return 
`BLOCK_CANNOT_MATCH` only if all the related pages are dictionary encoded. 
Otherwise you return `BLOCK_MIGHT_MATCH` since you don't know anything about 
the not dictionary encoded pages based on the dictionary.
   





> 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