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

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

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


##########
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, @yabola, let me summarize my thoughts because I'm afraid I didn't 
describe them well before. Please correct me if I'm wrong.
   In this PR we are trying to optimize the logic of 
[RowGroupFilter](https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/RowGroupFilter.java#L102-L112).
 The problem with the current implementation is we step forward to the next 
filter even if the previous one would prove that a value we are searching for 
is actually (not possibly) in the row group. The idea is to introduce 
`BLOCK_MUST_MATCH` and if this is returned by any of the filters we would not 
step forward to the next filter and add the row group to the list (do not drop 
it). We currently have 3 row group level filters.
   * `StatisticsFilter`: Because of the lower/upper bound issue we cannot 
really improve this (except for the specific case when min=max)
   * `DictionaryFilter`: We only can improve (?) the case when not all the 
pages are dictionary encoded because otherwise we would not have a Bloom filter 
so we won't step to the next filter anyway. So the dilemma is whether it worth 
to load the dictionary (which is potentially large since not all the values in 
the column chunk can fit in it) or is it better to use Bloom filter only. (The 
latter one is the current implementation.)
   * `BloomFilterImpl`: By nature we do not have a `BLOCK_MUST_MATCH` option.





> 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