[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-04-22 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/dictionarylevel/DictionaryFilter.java:
##
@@ -52,14 +53,24 @@ public class DictionaryFilter implements 
FilterPredicate.Visitor {
   private static final boolean BLOCK_MIGHT_MATCH = false;
   private static final boolean BLOCK_CANNOT_MATCH = true;
 
-  public static boolean canDrop(FilterPredicate pred, 
List columns, DictionaryPageReadStore dictionaries) {
+  public static boolean canDrop(FilterPredicate pred, 
List columns,
+DictionaryPageReadStore dictionaries, AtomicBoolean canExactlyDetermine) {

Review Comment:
   This is a public method. We are changing the signature now. Can you add 
another one to support callers that are still calling the old one?





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-04-22 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/dictionarylevel/DictionaryFilter.java:
##
@@ -52,14 +53,24 @@ public class DictionaryFilter implements 
FilterPredicate.Visitor {
   private static final boolean BLOCK_MIGHT_MATCH = false;
   private static final boolean BLOCK_CANNOT_MATCH = true;
 
-  public static boolean canDrop(FilterPredicate pred, 
List columns, DictionaryPageReadStore dictionaries) {
+  public static boolean canDrop(FilterPredicate pred, 
List columns,
+DictionaryPageReadStore dictionaries, AtomicBoolean canExactlyDetermine) {

Review Comment:
   This is a public method. We are changing the signature now. Can you add 
another one to support callers that are still calling the old one?





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-03-07 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

yabola commented on PR #1023:
URL: https://github.com/apache/parquet-mr/pull/1023#issuecomment-1458436214

   > Thanks @yabola for coming up with this idea. Let's continue the discussion 
about the BloomFilter building idea in the jira.
   > 
   > Meanwhile, I've been thinking about the actual problem as well. Currently, 
for row group filtering we are checking the min/max values first which is 
correct since this is the most fast thing to do. Then the dictionary and then 
the bloom filter. The ordering of the latter two is not obvious to me in every 
scenarios. At the time of filtering we did not start reading the actual row 
group so there is no advantage in I/O to read the dictionary first. 
Furthermore, searching something in the bloom filter is much faster than in the 
dictionary. And the size of the bloom filter is probably less than the size of 
the dictionary. Though, it would require some measurements to prove if it is a 
good idea to get the bloom filter before the dictionary. What do you think?
   
   It is a good idea to adjust filter order and prefer the use of lighter 
filters first to judge.
   But I have some concern (not sure if it is correct): 
   In parquet dictionary will generate only in low-base data( see 
`parquet.dictionary.page.size` 1 MB), and BloomFilter is usually used in high 
base columns(?) . So ideally only one of these two will be used(?)
   
   And ideally we should only use one of these two (don't judge both of them). 
If there is a BloomFilter and filter is `=` or `in`, only use the BloomFilter , 
otherwise use 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)


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-03-07 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

wgtmac commented on PR #1023:
URL: https://github.com/apache/parquet-mr/pull/1023#issuecomment-1458274839

   > Thanks @yabola for coming up with this idea. Let's continue the discussion 
about the BloomFilter building idea in the jira.
   > 
   > Meanwhile, I've been thinking about the actual problem as well. Currently, 
for row group filtering we are checking the min/max values first which is 
correct since this is the most fast thing to do. Then the dictionary and then 
the bloom filter. The ordering of the latter two is not obvious to me in every 
scenarios. At the time of filtering we did not start reading the actual row 
group so there is no advantage in I/O to read the dictionary first. 
Furthermore, searching something in the bloom filter is much faster than in the 
dictionary. And the size of the bloom filter is probably less than the size of 
the dictionary. Though, it would require some measurements to prove if it is a 
good idea to get the bloom filter before the dictionary. What do you think?
   
   What I did in production is to issue async I/Os of dictionaries (if all data 
pages are dictionary-encoded in that column chunk and the dictionary is not 
big) and bloom filters of selected row groups in advance. The reason is to 
eliminate blocking I/O when pushing down the predicates. However, the parquet 
specs only records the offset to bloom filter. So I also added the length of 
each bloom filter in the key value metadata section (probably a good reason to 
add to the specs as well?)




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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-03-07 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

gszadovszky commented on PR #1023:
URL: https://github.com/apache/parquet-mr/pull/1023#issuecomment-1457722446

   Thanks @yabola for coming up with this idea. Let's continue the discussion 
about the BloomFilter building idea in the jira.
   
   Meanwhile, I've been thinking about the actual problem as well. Currently, 
for row group filtering we are checking the min/max values first which is 
correct since this is the most fast thing to do. Then the dictionary and then 
the bloom filter. The ordering of the latter two is not obvious to me in every 
scenarios. At the time of filtering we did not start reading the actual row 
group so there is no advantage in I/O to read the dictionary first. 
Furthermore, searching something in the bloom filter is much faster than in the 
dictionary. And the size of the bloom filter is probably less than the size of 
the dictionary. Though, it would require some measurements to prove if it is a 
good idea to get the bloom filter before the dictionary. What do you think?




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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-03-06 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

yabola commented on PR #1023:
URL: https://github.com/apache/parquet-mr/pull/1023#issuecomment-1457669848

   @wgtmac @gszadovszky 
   I have a proposal to  automatically build BloomFilter with a more precise 
size. I create a jira https://issues.apache.org/jira/browse/PARQUET-2254 and  I 
hope to get your opinions, thank you.
   
   > Now the usage is to specify the size, and then build BloomFilter. In 
general scenarios, it is actually not sure how much the distinct value is.
   If BloomFilter can be automatically generated according to the data, the 
file size can be reduced and the reading efficiency can also be improved.
   
   I have an idea that the user can specify a maximum BloomFilter filter size, 
then we build several BloomFilter at the same time, we can use the largest 
BloomFilter as a counting tool( If there is no hit when inserting a value, the 
counter will be +1, of course this may be imprecise but enough)
   Then at the end of the write, choose a BloomFilter of a more appropriate 
size when the file is finally written.
   
   I want to implement this feature and




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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-03-05 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

yabola closed pull request #1039: PARQUET-2237 Improve performance by skipping 
BloomFilter  when column has a dictionary filter
URL: https://github.com/apache/parquet-mr/pull/1039




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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-03-05 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

yabola commented on PR #1039:
URL: https://github.com/apache/parquet-mr/pull/1039#issuecomment-1455273316

   The implementation of `and` `or` in this PR is incorrect, I want to change 
to the previous implementation in https://github.com/apache/parquet-mr/pull/1023




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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-03-02 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

yabola commented on PR #1039:
URL: https://github.com/apache/parquet-mr/pull/1039#issuecomment-1452828333

   @wgtmac @gszadovszky If you have time, please take a look, thank you




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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-22 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2237?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=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 > Boolean visit(Lt 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)


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-22 Thread ASF GitHub Bot (Jira)


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

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 > Boolean visit(Lt 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 worth doing it. 
   
   





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-22 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2237?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=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 > Boolean visit(Lt 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)


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-22 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##
@@ -289,8 +320,14 @@ public > Boolean visit(Lt 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:
   > We have dictionary encoding but not for all the pages. We also have Bloom 
filter.
   
   Yes, that's true.
   
   > 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?
   
   In this case, the dictionary will not be read via `expandDictionary(meta)` 
by `DictionaryFilter` if `hasNonDictionaryPages(meta)` returns true and will 
not make performance worse. e.g. 
https://github.com/apache/parquet-mr/blob/261f7d2679407c833545b56f4c85a4ae8b5c9ed4/parquet-hadoop/src/main/java/org/apache/parquet/filter2/dictionarylevel/DictionaryFilter.java#L388
   
   > 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.
   
   It is difficult to make the trade-off here. If we only have one predicate, 
then the dictionary will read any way, either by the DictionaryFilter or by 
reading the data later if the row group cannot be dropped. However, if we have 
other predicates that can drop the row group, then reading the dictionary here 
by DictionaryFilter is worthless.





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-22 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2237?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=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 > Boolean visit(Lt 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)


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-22 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##
@@ -289,8 +320,14 @@ public > Boolean visit(Lt 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 @gszadovszky I think it might be bug here because we use 
`PLAIN_DICTIONARY` in parquet v1? 
   Bloomfilter will avoid generating only when the dictionary types are all 
`RLE_DICTIONARY`? ( we should also add  `PLAIN_DICTIONARY`,  I can fix this bug 
in another PR and check more)

https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java#L884

I quick tested on it and found  
`EncodingStats#hasNonDictionaryEncodedPages` return false (means all pages are 
encoded) but `BloomFilterImpl` can still load `BloomFilter`.
 





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-22 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##
@@ -289,8 +320,14 @@ public > Boolean visit(Lt 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:
   > Good catch! I am not familiar with the old story. Does format v1 support 
bloom filter?
   
   Yes, and Spark use parquet v1 by default





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-22 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##
@@ -289,8 +320,14 @@ public > Boolean visit(Lt 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:
   > Good catch! I am not familiar with the old story. Does format v1 support 
bloom filter?
   
   Yes, Spark use parquet v1 by default





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-22 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##
@@ -289,8 +320,14 @@ public > Boolean visit(Lt 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 @gszadovszky I think it might be bug here because we use 
`PLAIN_DICTIONARY` in parquet v1? 
   Bloomfilter will avoid generating only when the dictionary types are all 
`RLE_DICTIONARY`? ( we should also add  `PLAIN_DICTIONARY`,  I can fix this bug 
in another PR and check more)

https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java#L884

I quick tested on it and found  
`EncodingStats#hasNonDictionaryEncodedPages` return false (means all pages are 
encoded) but `BloomFilterImpl` can still read `BloomFilter`.
 





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-22 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##
@@ -289,8 +320,14 @@ public > Boolean visit(Lt 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 @gszadovszky I think it might be bug here because we use 
`PLAIN_DICTIONARY` in parquet v1? 
   Bloomfilter will avoid generating only when the dictionary types are all 
`RLE_DICTIONARY`? ( we should also add  `PLAIN_DICTIONARY`,  I can fix this bug 
in another PR, I might have to check more)

https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java#L884

I quick tested on it and found  
`EncodingStats#hasNonDictionaryEncodedPages` return false (means all pages are 
encoded) but `BloomFilterImpl` can still read `BloomFilter`.
 





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-22 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##
@@ -289,8 +320,14 @@ public > Boolean visit(Lt 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:
   Good catch! I am not familiar with the old story. Does format v1 support 
bloom filter?





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-22 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##
@@ -289,8 +320,14 @@ public > Boolean visit(Lt 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 @gszadovszky I think it might be bug here because we use 
`PLAIN_DICTIONARY` in parquet v1? 
   Bloomfilter will avoid generating only when the dictionary types are all 
`RLE_DICTIONARY`? ( I can fix this bug in another PR, I might have to check 
more)

https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java#L884

I quick tested on it and found  
`EncodingStats#hasNonDictionaryEncodedPages` return false (means all pages are 
encoded) but `BloomFilterImpl` can still read `BloomFilter`.
 





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-22 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##
@@ -289,8 +320,14 @@ public > Boolean visit(Lt 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 @gszadovszky I think it might be bug here because we use 
`PLAIN_DICTIONARY` in parquet v1? 
   Bloomfilter will avoid generating only when the dictionary types are all 
`RLE_DICTIONARY`? ( I can fix this bug in another PR)

https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java#L884

I quick tested on it and found  
`EncodingStats#hasNonDictionaryEncodedPages` return false (means all pages are 
encoded) but `BloomFilterImpl` can still read `BloomFilter`
 





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-22 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##
@@ -289,8 +320,14 @@ public > Boolean visit(Lt 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 @gszadovszky I think it might be bug here because we use 
`PLAIN_DICTIONARY` in parquet v1? 
   Bloomfilter will avoid generating only when the dictionary types are all 
`RLE_DICTIONARY`? ( I can fix in another PR)

https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java#L884

I quick tested on it and found  
`EncodingStats#hasNonDictionaryEncodedPages` return false (means all pages are 
encoded) but `BloomFilterImpl` can still read `BloomFilter`
 





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-22 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##
@@ -289,8 +320,14 @@ public > Boolean visit(Lt 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 @gszadovszky I think it might be bug here because we use 
`PLAIN_DICTIONARY` in parquet v1? 
   Bloomfilter will avoid generating only when the dictionary type is all 
`RLE_DICTIONARY`? ( I can fix in another PR)

https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java#L884

I quick tested on it and found  
`EncodingStats#hasNonDictionaryEncodedPages` return false (means all pages are 
encoded) but `BloomFilterImpl` can still read `BloomFilter`
 





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-22 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##
@@ -289,8 +320,14 @@ public > Boolean visit(Lt 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 @gszadovszky I think it might be bug here because we use 
`PLAIN_DICTIONARY` in parquet v1? 
   Bloomfilter will avoid generating only when the dictionary type is all 
`RLE_DICTIONARY`?

https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java#L884





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-22 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##
@@ -289,8 +320,14 @@ public > Boolean visit(Lt 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:
   > We should not write bloom filters if all the pages are dictionary encoded 
anyway
   
   @gszadovszky CMIW, in the existing implementation, `BloomFilter` will still 
generate even if all pages are encoded have dictionaries.





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-22 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##
@@ -289,8 +320,14 @@ public > Boolean visit(Lt 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:
   I think @gszadovszky is correct.
   
   
https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java#L884





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-22 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##
@@ -289,8 +320,14 @@ public > Boolean visit(Lt 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:
   > We should not write bloom filters if all the pages are dictionary encoded 
anyway
   
   @gszadovszky CMIW, in the existing implementation, `BloomFilter` will still 
generate even if all pages are encoded have dictionaries.





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-22 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/PredicateEvaluation.java:
##
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.parquet.filter2.compat;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.parquet.filter2.predicate.FilterPredicate;
+import org.apache.parquet.filter2.predicate.Operators;
+
+/**
+ * Used in Filters to mark whether we should DROP the block if data matches 
the condition.
+ * If we cannot decide whether the block matches, it will be always safe to 
return BLOCK_MIGHT_MATCH.
+ * We use Boolean Object here to distinguish the value type, please do not 
modify it.
+ */
+public class PredicateEvaluation {
+  /* The block might match, but we cannot decide yet, will check in the other 
filters. */
+  public static final Boolean BLOCK_MIGHT_MATCH = new Boolean(false);
+  /* The block can match for sure. */
+  public static final Boolean BLOCK_MUST_MATCH = new Boolean(false);
+  /* The block can't match for sure */
+  public static final Boolean BLOCK_CANNOT_MATCH = new Boolean(true);
+
+  public static Boolean evaluateAnd(Operators.And and, 
FilterPredicate.Visitor predicate) {
+Boolean left = and.getLeft().accept(predicate);
+if (left == BLOCK_CANNOT_MATCH) {
+  // seems unintuitive to put an || not an && here but we can
+  // drop a chunk of records if we know that either the left or
+  // the right predicate agrees that no matter what we don't
+  // need this chunk.
+  return BLOCK_CANNOT_MATCH;
+}
+Boolean right = and.getRight().accept(predicate);
+if (right == BLOCK_CANNOT_MATCH) {
+  return BLOCK_CANNOT_MATCH;
+} else if (left == BLOCK_MUST_MATCH && right == BLOCK_MUST_MATCH) {

Review Comment:
   I didn't get your point here. @yabola 
   
   If the current expression is `A and B`, then following result applies 
regardless of other expressions:
   - A is BLOCK_MUST_MATCH and B is BLOCK_MUST_MATCH => BLOCK_MUST_MATCH
   - A is BLOCK_MUST_MATCH and B is BLOCK_MIGHT_MATCH => BLOCK_MUST_MATCH
   - A is BLOCK_MIGHT_MATCH and B is BLOCK_MUST_MATCH => BLOCK_MUST_MATCH
   - A is BLOCK_MIGHT_MATCH and B is BLOCK_MIGHT_MATCH => BLOCK_MIGHT_MATCH
   - A is BLOCK_CANNOT_MATCH or/and B is BLOCK_CANNOT_MATCH => 
BLOCK_CANNOT_MATCH



##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/RowGroupFilter.java:
##
@@ -97,21 +100,32 @@ public List 
visit(FilterCompat.FilterPredicateCompat filterPredic
 List filteredBlocks = new ArrayList();
 
 for (BlockMetaData block : blocks) {
-  boolean drop = false;
-
-  if(levels.contains(FilterLevel.STATISTICS)) {
-drop = StatisticsFilter.canDrop(filterPredicate, block.getColumns());
+  Boolean predicate = BLOCK_MIGHT_MATCH;
+  if (levels.contains(FilterLevel.STATISTICS)) {
+predicate = StatisticsFilter.predicate(filterPredicate, 
block.getColumns());
+if(isExactPredicate(predicate)) {

Review Comment:
   nit: name of `isExactPredicate` is a little bit unclear.





> 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 

[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-22 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##
@@ -289,8 +320,14 @@ public > Boolean visit(Lt 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 Maybe I didn't fully understand your point. Are you concerned 
about the performance or correctness? The `DictionaryFilter` is already 
decoding the dictionary to evaluate predicates like `lt` to return 
`BLOCK_CANNOT_MATCH`. This patch does not introduce more overhead but supports 
returning `BLOCK_MUST_MATCH` at the same time.





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-22 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##
@@ -289,8 +320,14 @@ public > Boolean visit(Lt 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, do you refering e.g. 
[this](https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/filter2/dictionarylevel/DictionaryFilter.java#L139-L143)?
 But this comment is only true if we are about to return `BLOCK_MIGHT_MATCH` or 
`BLOCK_CANNOT_MATCH`. If we want to potentially return `BLOCK_MUST_MATCH` we 
need to read the dictionary and check the values. So we might spend more time 
on dictionary filtering to potentially skip using bloom filters. But what if we 
don't have bloom filters? (We should not write bloom filters if all the pages 
are dictionary encoded anyway.)
   
   





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-22 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##
@@ -289,8 +320,14 @@ public > Boolean visit(Lt 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.
   
   CMIW, the `DictionaryFilter` is only enabled when all data pages are 
dictionary-encoded in a certain column chunk. So @yabola 's statement seems 
work to me.





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-22 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2237?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=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 > Boolean visit(Lt 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)


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-21 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##
@@ -289,8 +320,14 @@ public > Boolean visit(Lt 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:
   > Dictionary contains the very values from the pages themselves
   
   @gszadovszky do you think  if it is suitable to use `BLOCK_MUST_MATCH` when 
all page are encoded in `DictionaryFilter` in some more cases?
   For example,
   1. when search for data = x1 and dictionary has one element = x1  , then 
return `BLOCK_MUST_MATCH`
   2. when search for data > x1 and every elements in dictionary > x1  , then 
return `BLOCK_MUST_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)


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-21 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##
@@ -289,8 +320,14 @@ public > Boolean visit(Lt 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:
   > Dictionary contains the very values from the pages themselves
   
   @gszadovszky do you think  if it is suitable to use `BLOCK_MUST_MATCH` when 
all page are encoded in `DictionaryFilter` in some cases?
   For example,
   1. when search for data = x1 and dictionary has one element = x1  , then 
return `BLOCK_MUST_MATCH`
   2. when search for data > x1 and every elements in dictionary > x1  , then 
return `BLOCK_MUST_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)


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-21 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##
@@ -289,8 +320,14 @@ public > Boolean visit(Lt 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:
   > Dictionary contains the very values from the pages themselves
   @gszadovszky do you think it is suitable to use `BLOCK_MUST_MATCH` when all 
page are encoded in `DictionaryFilter`?
   For example,
   1. when search for data = x1 and dictionary has one element = x1  , then 
return `BLOCK_MUST_MATCH`
   2. when search for data > x1 and every elements in dictionary > x1  , then 
return `BLOCK_MUST_MATCH`
   
   



##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##
@@ -289,8 +320,14 @@ public > Boolean visit(Lt 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:
   > Dictionary contains the very values from the pages themselves
   
   @gszadovszky do you think it is suitable to use `BLOCK_MUST_MATCH` when all 
page are encoded in `DictionaryFilter`?
   For example,
   1. when search for data = x1 and dictionary has one element = x1  , then 
return `BLOCK_MUST_MATCH`
   2. when search for data > x1 and every elements in dictionary > x1  , then 
return `BLOCK_MUST_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)


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-21 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##
@@ -289,8 +320,14 @@ public > Boolean visit(Lt 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:
   FYI: for column index we specified it explicitly to allow shortening the 
min/max values (e.g. string are truncated to 64bytes by default). 
   The older min/max statistics for the row groups are trickier since it is not 
properly specified. The issue is if only one implementation of parquet in the 
wild would do the similar truncate we may loose data when filtering if we use 
this "aggressive optimization". 
   Dictionary is a completely different thing. It contains the very values from 
the pages themselves. It is not designed to be used for filtering (but we use 
anyway) but to save space.





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-21 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##
@@ -289,8 +321,16 @@ public > Boolean visit(Lt lt) {
 
 T value = lt.getValue();
 
-// drop if value <= min
-return stats.compareMinToValue(value) >= 0;
+

Review Comment:
   Minor: extra blank line.



##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##
@@ -325,8 +365,16 @@ public > Boolean visit(LtEq 
ltEq) {
 
 T value = ltEq.getValue();
 
-// drop if value < min
-return stats.compareMinToValue(value) > 0;
+

Review Comment:
   Minor: extra blank line.



##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/bloomfilterlevel/BloomFilterImpl.java:
##
@@ -28,26 +33,29 @@
 import org.slf4j.LoggerFactory;
 
 import org.apache.parquet.column.values.bloomfilter.BloomFilter;
+import org.apache.parquet.filter2.compat.PredicateEvaluation;
 import org.apache.parquet.filter2.predicate.FilterPredicate;
 import org.apache.parquet.filter2.predicate.Operators;
 import org.apache.parquet.filter2.predicate.UserDefinedPredicate;
 import org.apache.parquet.hadoop.BloomFilterReader;
 import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
 import org.apache.parquet.hadoop.metadata.ColumnPath;
 
-import static org.apache.parquet.Preconditions.checkNotNull;
-
 public class BloomFilterImpl implements FilterPredicate.Visitor{
   private static final Logger LOG = 
LoggerFactory.getLogger(BloomFilterImpl.class);
-  private static final boolean BLOCK_MIGHT_MATCH = false;
-  private static final boolean BLOCK_CANNOT_MATCH = true;
 
   private final Map columns = new 
HashMap();
 
-  public static boolean canDrop(FilterPredicate pred, 
List columns, BloomFilterReader bloomFilterReader) {

Review Comment:
   Minor: This line can remain unchanged if we move the `#predicate` down.



##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##
@@ -66,13 +72,17 @@
 // TODO: (https://issues.apache.org/jira/browse/PARQUET-38)
 public class StatisticsFilter implements FilterPredicate.Visitor {
 
-  private static final boolean BLOCK_MIGHT_MATCH = false;
-  private static final boolean BLOCK_CANNOT_MATCH = true;
-
-  public static boolean canDrop(FilterPredicate pred, 
List columns) {

Review Comment:
   Minor: This line can remain unchanged if we move the #predicate down.



##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/dictionarylevel/DictionaryFilter.java:
##
@@ -147,6 +172,9 @@ public > Boolean visit(Eq eq) {
   if (dictSet != null && !dictSet.contains(value)) {
 return BLOCK_CANNOT_MATCH;
   }
+  if (dictSet != null && dictSet.contains(value)) {
+return BLOCK_MUST_MATCH;
+  }

Review Comment:
   ```suggestion
 if(dictSet != null) {
   return dictSet.contains(value) ? BLOCK_MUST_MATCH || 
BLOCK_CANNOT_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)


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-21 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##
@@ -289,8 +320,14 @@ public > Boolean visit(Lt 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:
   I'm not aware of any implementation use lower/upper bounds instead of 
min/max values. But the specs does allow it to happen and `ColumnIndex` has 
stated about it 
[(link)](https://github.com/apache/parquet-format/blob/master/src/main/thrift/parquet.thrift#L956)
 explicitly as mentioned by @gszadovszky 
   
   I prefer to be conservative and make it easy to use:
   - Regard statistics as lower/upper bounds unless `min == max`.
   - Do not add any config for an aggressive optimization.





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-21 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##
@@ -289,8 +320,14 @@ public > Boolean visit(Lt 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 Thank you very much for your advice.
   But this will reduce much usage scenarios. From the code, `minMax` and 
`dictionary` is basically the minMax of real data. But the specification 
doesn't say.
   Can we add a config (default false, don't use `BLOCK_MUST_MATCH`) to 
optimize it?





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-21 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##
@@ -289,8 +320,14 @@ public > Boolean visit(Lt 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 Thank you very much for your advice.
   But this will reduce most usage scenarios. From the code, `minMax` and 
`dictionary` is basically the minMax of real data. But the specification 
doesn't say.
   Can we add a config (default false, don't use `BLOCK_MUST_MATCH`) to 
optimize it?





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-21 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##
@@ -289,8 +320,14 @@ public > Boolean visit(Lt 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:
   >  From the code, minMax and dictionary is basically the minMax of real 
data. But the specification doesn't say.
   
   IIUC, there may be other implementation libraries where min-max is not part 
of the row-group data set, because the specification does not define it 
clearly. Although I don't know of any library that does this.
   
   > Can we add a config (default false, don't use BLOCK_MUST_MATCH) to 
optimize it?
   
   From my point of view, it is doable to introduce such a config, but I am a 
little concerned that it will be difficult for users to understand.





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-21 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##
@@ -289,8 +320,14 @@ public > Boolean visit(Lt 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 Thank you very much for your advice.
   But this will reduce some usage scenarios. From the code, `minMax` and 
`dictionary` is basically the minMax of real data. But the specification 
doesn't say.
   Can we add a config (default false, don't use `BLOCK_MUST_MATCH`) to 
optimize it?





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-21 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##
@@ -289,8 +320,14 @@ public > Boolean visit(Lt 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 Thank you for your review.
   In the original implementation, `BLOCK_CANNOT_MATCH ` can be judged using 
minMax. 
   So if we follow the specification, can we use minMax as the data result of 
an enlarged range? So we can accurately judge that when the data is not in this 
range.





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-21 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2237?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=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_r111478


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##
@@ -289,8 +320,14 @@ public > Boolean visit(Lt 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)


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-21 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##
@@ -289,8 +320,14 @@ public > Boolean visit(Lt 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 Thank you for your review!
   In the original implementation, `BLOCK_MUST_MATCH` can be judged using 
minMax. 
   So if we follow the specification, can we use minMax as the data result of 
an enlarged range? So we can accurately judge that when the data is not in this 
range.



##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##
@@ -289,8 +320,14 @@ public > Boolean visit(Lt 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 Thank you for your review.
   In the original implementation, `BLOCK_MUST_MATCH` can be judged using 
minMax. 
   So if we follow the specification, can we use minMax as the data result of 
an enlarged range? So we can accurately judge that when the data is not in this 
range.





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-21 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##
@@ -289,8 +320,14 @@ public > Boolean visit(Lt 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:
   In fact, what I wanted to do at the beginning was to avoid the use of 
`BloomFilter` through `minMax` and `dictionary`(if column has) as much as 
possible, because the `minMax` and `dictionary` are more accurate and 
`BloomFilter` may cost time and memory.





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-21 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##
@@ -289,8 +320,14 @@ public > Boolean visit(Lt 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:
   In fact, what I wanted to do at the beginning was to avoid the use of 
`BloomFilter` through `minMax` and `dictionary`(if column have) as much as 
possible, because the former is more accurate and `BloomFilter` may cost time 
and memory.





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-21 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##
@@ -289,8 +320,14 @@ public > Boolean visit(Lt 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:
   In fact, what I wanted to do at the beginning was to avoid the use of 
`BloomFilter` through `minMax` and `dictionary`(if column has) as much as 
possible, because the former is more accurate and `BloomFilter` may cost time 
and memory.





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-21 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##
@@ -289,8 +320,14 @@ public > Boolean visit(Lt 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 Thank you for your reply!
   In the original implementation, `BLOCK_MUST_MATCH` can be judged using 
minMax. 
   So if we follow the specification, can we use minMax as the data result of 
an enlarged range? So we can accurately judge that when the data is not in this 
range.





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-21 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##
@@ -289,8 +320,14 @@ public > Boolean visit(Lt 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 Thank you for your reply!
   In the original implementation, `BLOCK_MUST_MATCH` can be judged using 
minMax. 
   So if we follow the specification, can we using minMax as the data result of 
an enlarged range. So we can accurately judge that when the data is not in this 
range.





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-21 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##
@@ -289,8 +320,14 @@ public > Boolean visit(Lt 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 Thank you for your reply!
   In the original implementation, `BLOCK_MUST_MATCH` can be judged using 
minMax. 
   So if we follow the specification, can we use minMax as the data result of 
an enlarged range. So we can accurately judge that when the data is not in this 
range.





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-21 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##
@@ -289,8 +320,14 @@ public > Boolean visit(Lt 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, do you aware of any implementations where the min/max values of the 
row group statistics are used this way? Unfortunately, the specification does 
not say anything about the min or max values has to be part of the dataset or 
not. The safe side would be to not to rely on this requirement. (For column 
index statistics we have defined that the related min/max values do not need to 
be part of the pages but it is not relevant here.)





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-20 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

yabola commented on PR #1023:
URL: https://github.com/apache/parquet-mr/pull/1023#issuecomment-1437950981

   @gszadovszky @shangxinli If you have time, please also take a look, thanks~




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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-20 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/PredicateEvaluation.java:
##
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.parquet.filter2.compat;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.parquet.filter2.predicate.FilterPredicate;
+import org.apache.parquet.filter2.predicate.Operators;
+
+/**
+ * Used in Filters to mark whether we should DROP the block if data matches 
the condition.
+ * If we cannot decide whether the block matches, it will be always safe to 
return BLOCK_MIGHT_MATCH.
+ * We use Boolean Object here to distinguish the value type, please do not 
modify it.
+ */
+public class PredicateEvaluation {
+  /* The block might match, but we cannot decide yet, will check in the other 
filters. */
+  public static final Boolean BLOCK_MIGHT_MATCH = new Boolean(false);
+  /* The block can match for sure. */
+  public static final Boolean BLOCK_MUST_MATCH = new Boolean(false);
+  /* The block can't match for sure */
+  public static final Boolean BLOCK_CANNOT_MATCH = new Boolean(true);
+
+  public static Boolean evaluateAnd(Operators.And and, 
FilterPredicate.Visitor predicate) {
+Boolean left = and.getLeft().accept(predicate);
+if (left == BLOCK_CANNOT_MATCH) {
+  // seems unintuitive to put an || not an && here but we can
+  // drop a chunk of records if we know that either the left or
+  // the right predicate agrees that no matter what we don't
+  // need this chunk.
+  return BLOCK_CANNOT_MATCH;
+}
+Boolean right = and.getRight().accept(predicate);
+if (right == BLOCK_CANNOT_MATCH) {
+  return BLOCK_CANNOT_MATCH;
+} else if (left == BLOCK_MUST_MATCH && right == BLOCK_MUST_MATCH) {

Review Comment:
   if left is `BLOCK_MUST_MATCH` , right is  `BLOCK_MIGHT_MATCH` , left & right 
should be `BLOCK_MIGHT_MATCH`.
   Because in the next filter may let right be `BLOCK_CANNOT_MATCH ` and we 
should drop it.
   
   And I add new 
[UT](https://github.com/apache/parquet-mr/pull/1023/files#diff-8915e6fa23018e02c2e79a3f6cc5078a8882f8031022dbdde217fe9bf1d908afR143)
   In  `StatisticsFilter` left might match (but can't match in 
DictionaryFilter),  right must match -> return might match in StatisticsFilter, 
 return can't match in DictionaryFilter
   
   





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-19 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

wgtmac commented on PR #1023:
URL: https://github.com/apache/parquet-mr/pull/1023#issuecomment-1436339171

   cc @zhongyujiang 
   Not sure if you are interested in reviewing this PR. 




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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-17 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/PredicateEvaluation.java:
##
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.parquet.filter2.compat;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.parquet.filter2.predicate.FilterPredicate;
+import org.apache.parquet.filter2.predicate.Operators;
+
+/**
+ * Used in Filters to mark whether we should DROP the block if data matches 
the condition.
+ * If we cannot decide whether the block matches, it will be always safe to 
return BLOCK_MIGHT_MATCH.
+ * We use Boolean Object here to distinguish the value type, please do not 
modify it.
+ */
+public class PredicateEvaluation {
+  /* The block might match, but we cannot decide yet, will check in the other 
filters. */
+  public static final Boolean BLOCK_MIGHT_MATCH = new Boolean(false);
+  /* The block can match for sure. */
+  public static final Boolean BLOCK_MUST_MATCH = new Boolean(false);
+  /* The block can't match for sure */
+  public static final Boolean BLOCK_CANNOT_MATCH = new Boolean(true);
+
+  public static Boolean evaluateAnd(Operators.And and, 
FilterPredicate.Visitor predicate) {
+Boolean left = and.getLeft().accept(predicate);
+if (left == BLOCK_CANNOT_MATCH) {
+  // seems unintuitive to put an || not an && here but we can
+  // drop a chunk of records if we know that either the left or
+  // the right predicate agrees that no matter what we don't
+  // need this chunk.
+  return BLOCK_CANNOT_MATCH;
+}
+Boolean right = and.getRight().accept(predicate);
+if (right == BLOCK_CANNOT_MATCH) {
+  return BLOCK_CANNOT_MATCH;
+} else if (left == BLOCK_MUST_MATCH && right == BLOCK_MUST_MATCH) {

Review Comment:
   if left is `BLOCK_MUST_MATCH` , right is  `BLOCK_MIGHT_MATCH` , left & right 
should be `BLOCK_MIGHT_MATCH`.
   Because in the next filter may let right may be `BLOCK_CANNOT_MATCH ` and we 
should drop it.
   
   And I add new 
[UT](https://github.com/apache/parquet-mr/pull/1023/files#diff-8915e6fa23018e02c2e79a3f6cc5078a8882f8031022dbdde217fe9bf1d908afR143)
   In  `StatisticsFilter` left might match (but can't match in 
DictionaryFilter),  right must match -> return might match in StatisticsFilter, 
 return can't match in DictionaryFilter
   
   





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-17 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/PredicateEvaluation.java:
##
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.parquet.filter2.compat;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.parquet.filter2.predicate.FilterPredicate;
+import org.apache.parquet.filter2.predicate.Operators;
+
+/**
+ * Used in Filters to mark whether we should DROP the block if data matches 
the condition.
+ * If we cannot decide whether the block matches, it will be always safe to 
return BLOCK_MIGHT_MATCH.
+ * We use Boolean Object here to distinguish the value type, please do not 
modify it.
+ */
+public class PredicateEvaluation {
+  /* The block might match, but we cannot decide yet, will check in the other 
filters. */
+  public static final Boolean BLOCK_MIGHT_MATCH = new Boolean(false);
+  /* The block can match for sure. */
+  public static final Boolean BLOCK_MUST_MATCH = new Boolean(false);
+  /* The block can't match for sure */
+  public static final Boolean BLOCK_CANNOT_MATCH = new Boolean(true);
+
+  public static Boolean evaluateAnd(Operators.And and, 
FilterPredicate.Visitor predicate) {
+Boolean left = and.getLeft().accept(predicate);
+if (left == BLOCK_CANNOT_MATCH) {
+  // seems unintuitive to put an || not an && here but we can
+  // drop a chunk of records if we know that either the left or
+  // the right predicate agrees that no matter what we don't
+  // need this chunk.
+  return BLOCK_CANNOT_MATCH;
+}
+Boolean right = and.getRight().accept(predicate);
+if (right == BLOCK_CANNOT_MATCH) {
+  return BLOCK_CANNOT_MATCH;
+} else if (left == BLOCK_MUST_MATCH && right == BLOCK_MUST_MATCH) {

Review Comment:
   if left is `BLOCK_MUST_MATCH` , right is  `BLOCK_MIGHT_MATCH` , left & right 
should be `BLOCK_MIGHT_MATCH`.
   Because in the next filter may right may be `BLOCK_CANNOT_MATCH ` and we 
should drop it.
   
   And I add new 
[UT](https://github.com/apache/parquet-mr/pull/1023/files#diff-8915e6fa23018e02c2e79a3f6cc5078a8882f8031022dbdde217fe9bf1d908afR143)
   In  `StatisticsFilter` left might match (but can't match in 
DictionaryFilter),  right must match -> return might match in StatisticsFilter, 
 return can't match in DictionaryFilter
   
   





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-17 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/PredicateEvaluation.java:
##
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.parquet.filter2.compat;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.parquet.filter2.predicate.FilterPredicate;
+import org.apache.parquet.filter2.predicate.Operators;
+
+/**
+ * Used in Filters to mark whether we should DROP the block if data matches 
the condition.
+ * If we cannot decide whether the block matches, it will be always safe to 
return BLOCK_MIGHT_MATCH.
+ * We use Boolean Object here to distinguish the value type, please do not 
modify it.
+ */
+public class PredicateEvaluation {
+  /* The block might match, but we cannot decide yet, will check in the other 
filters. */
+  public static final Boolean BLOCK_MIGHT_MATCH = new Boolean(false);
+  /* The block can match for sure. */
+  public static final Boolean BLOCK_MUST_MATCH = new Boolean(false);
+  /* The block can't match for sure */
+  public static final Boolean BLOCK_CANNOT_MATCH = new Boolean(true);
+
+  public static Boolean evaluateAnd(Operators.And and, 
FilterPredicate.Visitor predicate) {
+Boolean left = and.getLeft().accept(predicate);
+if (left == BLOCK_CANNOT_MATCH) {
+  // seems unintuitive to put an || not an && here but we can
+  // drop a chunk of records if we know that either the left or
+  // the right predicate agrees that no matter what we don't
+  // need this chunk.
+  return BLOCK_CANNOT_MATCH;
+}
+Boolean right = and.getRight().accept(predicate);
+if (right == BLOCK_CANNOT_MATCH) {
+  return BLOCK_CANNOT_MATCH;
+} else if (left == BLOCK_MUST_MATCH && right == BLOCK_MUST_MATCH) {
+  // if left and right operation all must needs the block, then we must 
take the block
+  return BLOCK_MUST_MATCH;
+} else {
+  return BLOCK_MIGHT_MATCH;
+}
+  }
+
+  public static Boolean evaluateOr(Operators.Or or, 
FilterPredicate.Visitor predicate) {
+Boolean left = or.getLeft().accept(predicate);
+if (left == BLOCK_MUST_MATCH) {
+  // if left or right operation must need the block, then we must take the 
block
+  return BLOCK_MUST_MATCH;
+}
+Boolean right = or.getRight().accept(predicate);
+if (right == BLOCK_MUST_MATCH) {
+  // if left or right operation must need the block, then we must take the 
block
+  return BLOCK_MUST_MATCH;
+} else if (left == BLOCK_CANNOT_MATCH && right == BLOCK_CANNOT_MATCH) {
+  // seems unintuitive to put an && not an || here

Review Comment:
   done





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-17 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/PredicateEvaluation.java:
##
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.parquet.filter2.compat;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.parquet.filter2.predicate.FilterPredicate;
+import org.apache.parquet.filter2.predicate.Operators;
+
+/**
+ * Used in Filters to mark whether we should DROP the block if data matches 
the condition.
+ * If we cannot decide whether the block matches, it will be always safe to 
return BLOCK_MIGHT_MATCH.
+ * We use Boolean Object here to distinguish the value type, please do not 
modify it.
+ */
+public class PredicateEvaluation {
+  /* The block might match, but we cannot decide yet, will check in the other 
filters. */
+  public static final Boolean BLOCK_MIGHT_MATCH = new Boolean(false);
+  /* The block can match for sure. */
+  public static final Boolean BLOCK_MUST_MATCH = new Boolean(false);
+  /* The block can't match for sure */
+  public static final Boolean BLOCK_CANNOT_MATCH = new Boolean(true);
+
+  public static Boolean evaluateAnd(Operators.And and, 
FilterPredicate.Visitor predicate) {
+Boolean left = and.getLeft().accept(predicate);
+if (left == BLOCK_CANNOT_MATCH) {
+  // seems unintuitive to put an || not an && here but we can
+  // drop a chunk of records if we know that either the left or
+  // the right predicate agrees that no matter what we don't
+  // need this chunk.
+  return BLOCK_CANNOT_MATCH;
+}
+Boolean right = and.getRight().accept(predicate);
+if (right == BLOCK_CANNOT_MATCH) {
+  return BLOCK_CANNOT_MATCH;
+} else if (left == BLOCK_MUST_MATCH && right == BLOCK_MUST_MATCH) {

Review Comment:
   if left is `BLOCK_MUST_MATCH` , right is might `BLOCK_MIGHT_MATCH` , left & 
right should be `BLOCK_MIGHT_MATCH`.
   Because in the next filter may right may be `BLOCK_CANNOT_MATCH ` and we 
should drop it.
   
   And I add new 
[UT](https://github.com/apache/parquet-mr/pull/1023/files#diff-8915e6fa23018e02c2e79a3f6cc5078a8882f8031022dbdde217fe9bf1d908afR143)
   In  `StatisticsFilter` left might match (but can't match in 
DictionaryFilter),  right must match -> return might match in StatisticsFilter, 
 return can't match in DictionaryFilter
   
   





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-17 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/PredicateEvaluation.java:
##
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.parquet.filter2.compat;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.parquet.filter2.predicate.FilterPredicate;
+import org.apache.parquet.filter2.predicate.Operators;
+
+/**
+ * Used in Filters to mark whether we should DROP the block if data matches 
the condition.
+ * If we cannot decide whether the block matches, it will be always safe to 
return BLOCK_MIGHT_MATCH.
+ * We use Boolean Object here to distinguish the value type, please do not 
modify it.
+ */
+public class PredicateEvaluation {
+  /* The block might match, but we cannot decide yet, will check in the other 
filters. */
+  public static final Boolean BLOCK_MIGHT_MATCH = new Boolean(false);
+  /* The block can match for sure. */
+  public static final Boolean BLOCK_MUST_MATCH = new Boolean(false);
+  /* The block can't match for sure */
+  public static final Boolean BLOCK_CANNOT_MATCH = new Boolean(true);
+
+  public static Boolean evaluateAnd(Operators.And and, 
FilterPredicate.Visitor predicate) {
+Boolean left = and.getLeft().accept(predicate);
+if (left == BLOCK_CANNOT_MATCH) {
+  // seems unintuitive to put an || not an && here but we can
+  // drop a chunk of records if we know that either the left or
+  // the right predicate agrees that no matter what we don't
+  // need this chunk.
+  return BLOCK_CANNOT_MATCH;
+}
+Boolean right = and.getRight().accept(predicate);
+if (right == BLOCK_CANNOT_MATCH) {
+  return BLOCK_CANNOT_MATCH;
+} else if (left == BLOCK_MUST_MATCH && right == BLOCK_MUST_MATCH) {

Review Comment:
   if left is `BLOCK_MUST_MATCH` , right is might `BLOCK_MIGHT_MATCH` , left & 
right should be `BLOCK_MIGHT_MATCH`.
   Because in the next filter may right may be `BLOCK_CANNOT_MATCH ` and we 
should drop it.
   
   And I add new 
[UT](https://github.com/apache/parquet-mr/pull/1023/files#diff-8915e6fa23018e02c2e79a3f6cc5078a8882f8031022dbdde217fe9bf1d908afR143)
   
   
   





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-17 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/PredicateEvaluation.java:
##
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.parquet.filter2.compat;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.parquet.filter2.predicate.FilterPredicate;
+import org.apache.parquet.filter2.predicate.Operators;
+
+/**
+ * Used in Filters to mark whether we should DROP the block if data matches 
the condition.
+ * If we cannot decide whether the block matches, it will be always safe to 
return BLOCK_MIGHT_MATCH.
+ * We use Boolean Object here to distinguish the value type, please do not 
modify it.
+ */
+public class PredicateEvaluation {
+  /* The block might match, but we cannot decide yet, will check in the other 
filters. */
+  public static final Boolean BLOCK_MIGHT_MATCH = new Boolean(false);
+  /* The block can match for sure. */
+  public static final Boolean BLOCK_MUST_MATCH = new Boolean(false);
+  /* The block can't match for sure */
+  public static final Boolean BLOCK_CANNOT_MATCH = new Boolean(true);
+
+  public static Boolean evaluateAnd(Operators.And and, 
FilterPredicate.Visitor predicate) {
+Boolean left = and.getLeft().accept(predicate);
+if (left == BLOCK_CANNOT_MATCH) {
+  // seems unintuitive to put an || not an && here but we can
+  // drop a chunk of records if we know that either the left or
+  // the right predicate agrees that no matter what we don't
+  // need this chunk.
+  return BLOCK_CANNOT_MATCH;
+}
+Boolean right = and.getRight().accept(predicate);
+if (right == BLOCK_CANNOT_MATCH) {
+  return BLOCK_CANNOT_MATCH;
+} else if (left == BLOCK_MUST_MATCH && right == BLOCK_MUST_MATCH) {

Review Comment:
   if left is `BLOCK_MUST_MATCH` , right is might `BLOCK_MIGHT_MATCH` , left & 
right should be `BLOCK_MIGHT_MATCH`.
   Because in the next filter may right may be `BLOCK_CANNOT_MATCH ` and we 
should drop it
   





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-17 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/PredicateEvaluation.java:
##
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.parquet.filter2.compat;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.parquet.filter2.predicate.FilterPredicate;
+import org.apache.parquet.filter2.predicate.Operators;
+
+/**
+ * Used in Filters to mark whether we should DROP the block if data matches 
the condition.
+ * If we cannot decide whether the block matches, it will be always safe to 
return BLOCK_MIGHT_MATCH.
+ * We use Boolean Object here to distinguish the value type, please do not 
modify it.
+ */
+public class PredicateEvaluation {
+  /* The block might match, but we cannot decide yet, will check in the other 
filters. */
+  public static final Boolean BLOCK_MIGHT_MATCH = new Boolean(false);
+  /* The block can match for sure. */
+  public static final Boolean BLOCK_MUST_MATCH = new Boolean(false);
+  /* The block can't match for sure */
+  public static final Boolean BLOCK_CANNOT_MATCH = new Boolean(true);
+
+  public static Boolean evaluateAnd(Operators.And and, 
FilterPredicate.Visitor predicate) {
+Boolean left = and.getLeft().accept(predicate);
+if (left == BLOCK_CANNOT_MATCH) {
+  // seems unintuitive to put an || not an && here but we can

Review Comment:
   Thanks for your review, I update comments and add more UT for `And` `Or`





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-17 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/PredicateEvaluation.java:
##
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.parquet.filter2.compat;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.parquet.filter2.predicate.FilterPredicate;
+import org.apache.parquet.filter2.predicate.Operators;
+
+/**
+ * Used in Filters to mark whether we should DROP the block if data matches 
the condition.
+ * If we cannot decide whether the block matches, it will be always safe to 
return BLOCK_MIGHT_MATCH.
+ * We use Boolean Object here to distinguish the value type, please do not 
modify it.
+ */
+public class PredicateEvaluation {
+  /* The block might match, but we cannot decide yet, will check in the other 
filters. */
+  public static final Boolean BLOCK_MIGHT_MATCH = new Boolean(false);
+  /* The block can match for sure. */
+  public static final Boolean BLOCK_MUST_MATCH = new Boolean(false);
+  /* The block can't match for sure */
+  public static final Boolean BLOCK_CANNOT_MATCH = new Boolean(true);
+
+  public static Boolean evaluateAnd(Operators.And and, 
FilterPredicate.Visitor predicate) {
+Boolean left = and.getLeft().accept(predicate);
+if (left == BLOCK_CANNOT_MATCH) {
+  // seems unintuitive to put an || not an && here but we can

Review Comment:
   thanks for your review,  added more UT for `And` `Or` to cover





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-15 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/PredicateEvaluation.java:
##
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.parquet.filter2.compat;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.parquet.filter2.predicate.FilterPredicate;
+import org.apache.parquet.filter2.predicate.Operators;
+
+/**
+ * Used in Filters to mark whether we should DROP the block if data matches 
the condition.
+ * If we cannot decide whether the block matches, it will be always safe to 
return BLOCK_MIGHT_MATCH.
+ * We use Boolean Object here to distinguish the value type, please do not 
modify it.
+ */
+public class PredicateEvaluation {
+  /* The block might match, but we cannot decide yet, will check in the other 
filters. */
+  public static final Boolean BLOCK_MIGHT_MATCH = new Boolean(false);
+  /* The block can match for sure. */
+  public static final Boolean BLOCK_MUST_MATCH = new Boolean(false);
+  /* The block can't match for sure */
+  public static final Boolean BLOCK_CANNOT_MATCH = new Boolean(true);
+
+  public static Boolean evaluateAnd(Operators.And and, 
FilterPredicate.Visitor predicate) {
+Boolean left = and.getLeft().accept(predicate);
+if (left == BLOCK_CANNOT_MATCH) {
+  // seems unintuitive to put an || not an && here but we can

Review Comment:
   The comment does not match the code now.



##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/PredicateEvaluation.java:
##
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.parquet.filter2.compat;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.parquet.filter2.predicate.FilterPredicate;
+import org.apache.parquet.filter2.predicate.Operators;
+
+/**
+ * Used in Filters to mark whether we should DROP the block if data matches 
the condition.
+ * If we cannot decide whether the block matches, it will be always safe to 
return BLOCK_MIGHT_MATCH.
+ * We use Boolean Object here to distinguish the value type, please do not 
modify it.
+ */
+public class PredicateEvaluation {
+  /* The block might match, but we cannot decide yet, will check in the other 
filters. */
+  public static final Boolean BLOCK_MIGHT_MATCH = new Boolean(false);
+  /* The block can match for sure. */
+  public static final Boolean BLOCK_MUST_MATCH = new Boolean(false);
+  /* The block can't match for sure */
+  public static final Boolean BLOCK_CANNOT_MATCH = new Boolean(true);
+
+  public static Boolean evaluateAnd(Operators.And and, 
FilterPredicate.Visitor predicate) {
+Boolean left = and.getLeft().accept(predicate);
+if (left == BLOCK_CANNOT_MATCH) {
+  // seems unintuitive to put an || not an && here but we can
+  // drop a chunk of records if we know that either the left or
+  // the right predicate agrees that no matter what we don't
+  // need this chunk.
+  return BLOCK_CANNOT_MATCH;
+}
+Boolean right = 

[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-15 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/PredicateEvaluation.java:
##
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.parquet.filter2.compat;
+
+import org.apache.parquet.filter2.predicate.FilterPredicate;
+import org.apache.parquet.filter2.predicate.Operators;
+
+/**
+ * Used in Filters to mark whether the block data matches the condition.
+ * If we cannot decide whether the block matches, it will be always safe to 
return BLOCK_MIGHT_MATCH.
+ *
+ * We use Boolean Object here to distinguish the value type, please do not 
modify it.
+ */
+public class PredicateEvaluation {
+  /* The block might match, but we cannot decide yet, will check in the other 
filters. */
+  public static final Boolean BLOCK_MIGHT_MATCH = new Boolean(false);
+  /* The block can match for sure. */
+  public static final Boolean BLOCK_MUST_MATCH = new Boolean(false);
+  /* The block can't match for sure */
+  public static final Boolean BLOCK_CANNOT_MATCH = new Boolean(true);
+
+  public static Boolean evaluateAnd(Operators.And and, 
FilterPredicate.Visitor predicate) {
+Boolean left = and.getLeft().accept(predicate);
+Boolean right = and.getRight().accept(predicate);
+if (left == BLOCK_CANNOT_MATCH || right == BLOCK_CANNOT_MATCH) {
+  return BLOCK_CANNOT_MATCH;
+} else if (left == BLOCK_MUST_MATCH && right == BLOCK_MUST_MATCH) {
+  // if left and right operation must need the block, then we must take 
the block
+  return BLOCK_MUST_MATCH;
+} else {
+  return BLOCK_MIGHT_MATCH;
+}
+  }
+
+  public static Boolean evaluateOr(Operators.Or or, 
FilterPredicate.Visitor predicate) {
+Boolean left = or.getLeft().accept(predicate);
+Boolean right = or.getRight().accept(predicate);
+if (left == BLOCK_CANNOT_MATCH && right == BLOCK_CANNOT_MATCH) {
+  return BLOCK_CANNOT_MATCH;
+} else if (left == BLOCK_MUST_MATCH || right == BLOCK_MUST_MATCH) {
+  // if left or right operation must need the block, then we must take the 
block
+  return BLOCK_MUST_MATCH;
+} else {
+  return BLOCK_MIGHT_MATCH;
+}
+  }
+
+  public static Boolean isDeterminedPredicate(Boolean predicate) {
+return predicate == BLOCK_MUST_MATCH || predicate == BLOCK_CANNOT_MATCH;
+  }
+
+  public static void checkPredicate(Boolean predicate) {

Review Comment:
   ```suggestion
 public static void checkPredicate(boolean predicate) {
   ```



##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/PredicateEvaluation.java:
##
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.parquet.filter2.compat;
+
+import org.apache.parquet.filter2.predicate.FilterPredicate;
+import org.apache.parquet.filter2.predicate.Operators;
+
+/**
+ * Used in Filters to mark whether the block data matches the condition.
+ * If we cannot decide whether the block matches, it will be always safe to 
return BLOCK_MIGHT_MATCH.
+ *
+ * We use Boolean Object here to distinguish the value type, please do not 

[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-12 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestRowGroupFilterExactly.java:
##
@@ -0,0 +1,303 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.parquet.hadoop;
+
+import static 
org.apache.parquet.filter2.compat.PredicateEvaluation.BLOCK_CANNOT_MATCH;
+import static 
org.apache.parquet.filter2.compat.PredicateEvaluation.BLOCK_MUST_MATCH;
+import static org.apache.parquet.filter2.predicate.FilterApi.and;
+import static org.apache.parquet.filter2.predicate.FilterApi.binaryColumn;
+import static org.apache.parquet.filter2.predicate.FilterApi.doubleColumn;
+import static org.apache.parquet.filter2.predicate.FilterApi.eq;
+import static org.apache.parquet.filter2.predicate.FilterApi.gt;
+import static org.apache.parquet.filter2.predicate.FilterApi.gtEq;
+import static org.apache.parquet.filter2.predicate.FilterApi.in;
+import static org.apache.parquet.filter2.predicate.FilterApi.longColumn;
+import static org.apache.parquet.filter2.predicate.FilterApi.lt;
+import static org.apache.parquet.filter2.predicate.FilterApi.ltEq;
+import static org.apache.parquet.filter2.predicate.FilterApi.notEq;
+import static org.apache.parquet.filter2.predicate.FilterApi.notIn;
+import static org.apache.parquet.filter2.predicate.FilterApi.or;
+import static org.apache.parquet.hadoop.ParquetFileWriter.Mode.OVERWRITE;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.junit.After;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.parquet.ParquetReadOptions;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.filter2.compat.FilterCompat;
+import org.apache.parquet.filter2.compat.PredicateEvaluation;
+import org.apache.parquet.filter2.predicate.FilterPredicate;
+import org.apache.parquet.filter2.recordlevel.PhoneBookWriter;
+import org.apache.parquet.hadoop.example.ExampleParquetWriter;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.io.api.Binary;
+
+import com.google.common.collect.Sets;
+
+@RunWith(Parameterized.class)
+public class TestRowGroupFilterExactly {
+  private final Path FILE = createTempFile();
+  private ParquetProperties.WriterVersion WRITER_VERSION;
+  private final Random RANDOM = new Random(42);
+  private final List DATA = 
Collections.unmodifiableList(generateData(1));
+  private final long MAX_ID = DATA.size() - 1;
+  private final long MIN_ID = 0;
+  private final TestPredicateEvaluation testEvaluation = new 
TestPredicateEvaluation();
+
+  @Parameterized.Parameters(name = "Run parquet version {index} ")
+  public static Collection params() {
+return Arrays.asList(
+  new Object[]{ParquetProperties.WriterVersion.PARQUET_1_0},
+  new Object[]{ParquetProperties.WriterVersion.PARQUET_2_0});
+  }
+
+  public TestRowGroupFilterExactly(ParquetProperties.WriterVersion 
WRITER_VERSION) throws IOException {
+this.WRITER_VERSION = WRITER_VERSION;
+deleteFile(FILE);
+writePhoneBookToFile(FILE, this.WRITER_VERSION);
+  }
+
+  @After
+  public void deleteFiles() throws IOException {
+deleteFile(FILE);
+testEvaluation.setTestExactPredicate(new 
ArrayList<>(Arrays.asList(BLOCK_MUST_MATCH, BLOCK_CANNOT_MATCH)));
+  }
+
+  @Test
+  public void testFiltering() throws IOException {
+
+Set existValues = new HashSet<>();
+

[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-12 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

yabola commented on PR #1023:
URL: https://github.com/apache/parquet-mr/pull/1023#issuecomment-1427053737

   @wgtmac @shangxinli I add new UT, if you have time, please take a look, 
thanks!




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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-10 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##
@@ -289,8 +320,14 @@ public > Boolean visit(Lt 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:
   I think  when `Statistics#hasNonNullValue` marks as true, minMax will be 
generated by the real data content, and it can represent the real data minMax ( 
when `Statistics#hasNonNullValue` is false, it has also been processed  before.)
   I think if we can use minMax to judge the `BLOCK_CANNOT_MATCH` ,  we can 
also judge the `BLOCK_MUST_MATCH ` in some case.
   





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-10 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##
@@ -289,8 +320,14 @@ public > Boolean visit(Lt 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:
   I think  when `Statistics#hasNonNullValue` marks as true, minMax will be 
generated by the real data content, and it can represent the real data minMax ( 
`Statistics#hasNonNullValue` has been processed before.)
   I think if we can use minMax to judge the `BLOCK_CANNOT_MATCH` ,  we can 
also judge the `BLOCK_MUST_MATCH ` in some case.
   





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-10 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##
@@ -289,8 +320,14 @@ public > Boolean visit(Lt 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:
   I think  when `Statistics#hasNonNullValue` marks as true, minMax will be 
generated by the data content, and it can represent the real data minMax ( 
`Statistics#hasNonNullValue` has been processed before.)
   I think if we can use minMax to judge the `BLOCK_CANNOT_MATCH` ,  we can 
also judge the `BLOCK_MUST_MATCH ` in some case.
   





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-10 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##
@@ -289,8 +320,14 @@ public > Boolean visit(Lt 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:
   I think  when `Statistics#hasNonNullValue` is true, it means that minMax is 
generated by the data content, so it can represent the real data minMax ( 
`Statistics#hasNonNullValue` has been processed before.)
   I think if we can use minMax to judge the `BLOCK_CANNOT_MATCH` ,  we can 
also judge the `BLOCK_MUST_MATCH ` in some case.
   





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-10 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/PredicateEvaluation.java:
##
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.parquet.filter2.compat;
+
+import org.apache.parquet.filter2.predicate.FilterPredicate;
+import org.apache.parquet.filter2.predicate.Operators;
+
+/**
+ * Used in Filters to mark whether the block data matches the condition.
+ * If we cannot decide whether the block matches, it will be always safe to 
return BLOCK_MIGHT_MATCH.
+ *
+ * We use Boolean Object here to distinguish the value type, please do not 
modify it.
+ */
+public class PredicateEvaluation {
+  /* The block might match, but we cannot decide yet, will check in the other 
filters. */
+  public static final Boolean BLOCK_MIGHT_MATCH = new Boolean(false);
+  /* The block can match for sure. */
+  public static final Boolean BLOCK_MUST_MATCH = new Boolean(false);
+  /* The block can't match for sure */
+  public static final Boolean BLOCK_CANNOT_MATCH = new Boolean(true);
+
+  public static Boolean evaluateAnd(Operators.And and, 
FilterPredicate.Visitor predicate) {
+Boolean left = and.getLeft().accept(predicate);

Review Comment:
   Yes, thanks





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-10 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

yabola commented on PR #1023:
URL: https://github.com/apache/parquet-mr/pull/1023#issuecomment-1425942666

   @wgtmac Sorry, `Boolean` type has to be used here, so that we can 
distinguish the `BLOCK_MIGHT_MATCH` and `BLOCK_MUST_MATCH`. This is example:
   ```
   Boolean b1 = new Boolean(true);
   Boolean b2 = new Boolean(true);
   boolean b3 = true;
   boolean b4 = true;
   
   assert b1 != b2;
   assert b1.equals(b2);
   assert b2 == b3 == b4;
   ```




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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-09 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/PredicateEvaluation.java:
##
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.parquet.filter2.compat;
+
+import org.apache.parquet.filter2.predicate.FilterPredicate;
+import org.apache.parquet.filter2.predicate.Operators;
+
+/**
+ * Used in Filters to mark whether the block data matches the condition.
+ * If we cannot decide whether the block matches, it will be always safe to 
return BLOCK_MIGHT_MATCH.
+ *
+ * We use Boolean Object here to distinguish the value type, please do not 
modify it.
+ */
+public class PredicateEvaluation {
+  /* The block might match, but we cannot decide yet, will check in the other 
filters. */
+  public static final Boolean BLOCK_MIGHT_MATCH = new Boolean(false);

Review Comment:
   ```suggestion
 public static final boolean BLOCK_MIGHT_MATCH = false;
   ```



##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/PredicateEvaluation.java:
##
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.parquet.filter2.compat;
+
+import org.apache.parquet.filter2.predicate.FilterPredicate;
+import org.apache.parquet.filter2.predicate.Operators;
+
+/**
+ * Used in Filters to mark whether the block data matches the condition.
+ * If we cannot decide whether the block matches, it will be always safe to 
return BLOCK_MIGHT_MATCH.
+ *
+ * We use Boolean Object here to distinguish the value type, please do not 
modify it.
+ */
+public class PredicateEvaluation {
+  /* The block might match, but we cannot decide yet, will check in the other 
filters. */
+  public static final Boolean BLOCK_MIGHT_MATCH = new Boolean(false);
+  /* The block can match for sure. */
+  public static final Boolean BLOCK_MUST_MATCH = new Boolean(false);
+  /* The block can't match for sure */
+  public static final Boolean BLOCK_CANNOT_MATCH = new Boolean(true);
+
+  public static Boolean evaluateAnd(Operators.And and, 
FilterPredicate.Visitor predicate) {
+Boolean left = and.getLeft().accept(predicate);
+Boolean right = and.getRight().accept(predicate);
+if (left == BLOCK_CANNOT_MATCH || right == BLOCK_CANNOT_MATCH) {
+  return BLOCK_CANNOT_MATCH;
+} else if (left == BLOCK_MUST_MATCH && right == BLOCK_MUST_MATCH) {
+  // if left and right operation must need the block, then we must take 
the block
+  return BLOCK_MUST_MATCH;
+} else {
+  return BLOCK_MIGHT_MATCH;
+}
+  }
+
+  public static Boolean evaluateOr(Operators.Or or, 
FilterPredicate.Visitor predicate) {
+Boolean left = or.getLeft().accept(predicate);
+Boolean right = or.getRight().accept(predicate);

Review Comment:
   ditto



##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/RowGroupFilter.java:
##
@@ -97,21 +100,37 @@ public List 
visit(FilterCompat.FilterPredicateCompat filterPredic
 List filteredBlocks = new ArrayList();
 
 for (BlockMetaData block : blocks) {
-  boolean drop = false;
+  

[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-08 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

yabola commented on PR #1023:
URL: https://github.com/apache/parquet-mr/pull/1023#issuecomment-1422839946

   E, If this way is not suitable, I can use the filter internal variable 
to represent and keep compatibility




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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-06 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

yabola commented on PR #1023:
URL: https://github.com/apache/parquet-mr/pull/1023#issuecomment-1420123553

   @wgtmac @shangxinli I thought of a way to avoid interface modification and 
distinguish by Boolean objects. Please take a look




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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-06 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

yabola commented on PR #1023:
URL: https://github.com/apache/parquet-mr/pull/1023#issuecomment-1419363481

   retest this please




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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-06 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

yabola commented on PR #1023:
URL: https://github.com/apache/parquet-mr/pull/1023#issuecomment-1419361716

   I thought of a way to use Boolean objects to distinguish the different 
types, It doesn't need to modify the visitor return type now.  But it seems 
Jenkins machine was broken?...
   




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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-06 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

shangxinli commented on PR #1023:
URL: https://github.com/apache/parquet-mr/pull/1023#issuecomment-1419254754

   +1, let's not modify the signature. 




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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-06 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

wgtmac commented on PR #1023:
URL: https://github.com/apache/parquet-mr/pull/1023#issuecomment-1418711576

   Unfortunately we cannot modify the signature of any public methods. My 
suggestion was to make the new enum serves as an internal state of the visitor 
(and probably use it to terminate evaluation early). Then add a new method to 
return the final state. Does it work?




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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-05 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

yabola commented on PR #1023:
URL: https://github.com/apache/parquet-mr/pull/1023#issuecomment-1418302885

   It seems code compatibility check error caused by modifying visitor return 
value... can we remove that restrictions ? or I should keep the code 
compatibility and add new flag to mark it?
   I will add more UT later




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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-05 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

yabola commented on PR #1023:
URL: https://github.com/apache/parquet-mr/pull/1023#issuecomment-1418295634

   > It seems code compatibility check error caused by modifying visitor return 
value... can we remove that restrictions ? I will add more UT later
   
   If we have to keep the code compatibility, can I add some flag in 
`DictionaryFilter` and other filters to complete such function?




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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-05 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

yabola commented on PR #1023:
URL: https://github.com/apache/parquet-mr/pull/1023#issuecomment-1418012997

   It seems code compatibility check error caused by modifying visitor return 
value...




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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-05 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

wgtmac commented on PR #1023:
URL: https://github.com/apache/parquet-mr/pull/1023#issuecomment-1418001192

   cc @gszadovszky @ggershinsky @shangxinli 




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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-05 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

yabola commented on PR #1023:
URL: https://github.com/apache/parquet-mr/pull/1023#issuecomment-1417985954

   please also cc @gszadovszky if you have time , I will add more UT and 
improve my codes.




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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-05 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/RowGroupFilter.java:
##
@@ -98,16 +99,19 @@ public List 
visit(FilterCompat.FilterPredicateCompat filterPredic
 
 for (BlockMetaData block : blocks) {
   boolean drop = false;
+  // Whether one filter can exactly determine the existence/nonexistence 
of the value.
+  // If true then we can skip the remaining filters to save time and space.
+  AtomicBoolean canExactlyDetermine = new AtomicBoolean(false);

Review Comment:
   I will change my implemention





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-05 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/RowGroupFilter.java:
##
@@ -98,16 +99,19 @@ public List 
visit(FilterCompat.FilterPredicateCompat filterPredic
 
 for (BlockMetaData block : blocks) {
   boolean drop = false;
+  // Whether one filter can exactly determine the existence/nonexistence 
of the value.
+  // If true then we can skip the remaining filters to save time and space.
+  AtomicBoolean canExactlyDetermine = new AtomicBoolean(false);

Review Comment:
   I will change my codes





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-05 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/test/java/org/apache/parquet/filter2/dictionarylevel/DictionaryFilterTest.java:
##
@@ -792,6 +793,16 @@ public void testInverseUdpMissingColumn() throws Exception 
{
   canDrop(LogicalInverseRewriter.rewrite(not(userDefined(fake, 
nullRejecting))), ccmd, dictionaries));
   }
 
+  @Test
+  public void testCanSkipOtherFilters() {

Review Comment:
   I will add more UT





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-05 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/RowGroupFilter.java:
##
@@ -98,16 +99,19 @@ public List 
visit(FilterCompat.FilterPredicateCompat filterPredic
 
 for (BlockMetaData block : blocks) {
   boolean drop = false;
+  // Whether one filter can exactly determine the existence/nonexistence 
of the value.
+  // If true then we can skip the remaining filters to save time and space.
+  AtomicBoolean canExactlyDetermine = new AtomicBoolean(false);

Review Comment:
   It used to be for the convenience of fetching the returned results. But I 
will change my codes in another implemention later





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


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-05 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

yabola commented on PR #1023:
URL: https://github.com/apache/parquet-mr/pull/1023#issuecomment-1417093967

   @wgtmac Thanks for review. I will address your comments and I updated my PR 
description to explain in more detail.




> 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
>
> Bloomfilter needs to load from filesystem, it may costs time and space. If we 
> can  exactly determine the existence/nonexistence of the value from other 
> filters , then we can avoid using Bloomfilter to Improve performance.
>  
> When the minMax values in  StatisticsFilter is same, we can exactly determine 
> the existence/nonexistence of the value.
> When we have page dictionaries, we can also determine the 
> existence/nonexistence of the value.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-04 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/RowGroupFilter.java:
##
@@ -98,16 +99,19 @@ public List 
visit(FilterCompat.FilterPredicateCompat filterPredic
 
 for (BlockMetaData block : blocks) {
   boolean drop = false;
+  // Whether one filter can exactly determine the existence/nonexistence 
of the value.
+  // If true then we can skip the remaining filters to save time and space.
+  AtomicBoolean canExactlyDetermine = new AtomicBoolean(false);

Review Comment:
   Why atomic?



##
parquet-hadoop/src/test/java/org/apache/parquet/filter2/dictionarylevel/DictionaryFilterTest.java:
##
@@ -792,6 +793,16 @@ public void testInverseUdpMissingColumn() throws Exception 
{
   canDrop(LogicalInverseRewriter.rewrite(not(userDefined(fake, 
nullRejecting))), ccmd, dictionaries));
   }
 
+  @Test
+  public void testCanSkipOtherFilters() {

Review Comment:
   The test looks a little bit insufficient. More kinds of predicates and 
compound predicates need to be covered. Also test of `RowGroupFilter` is 
missing.



##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/dictionarylevel/DictionaryFilter.java:
##
@@ -559,4 +591,12 @@ private static boolean 
hasNonDictionaryPages(ColumnChunkMetaData meta) {
   return true;
 }
   }
+
+  private > void markCanExactlyDetermine(Set 
dictSet) {
+if (dictSet == null) {
+  canExactlyDetermine = false;

Review Comment:
   It seems that `canExactlyDetermine` should use `OR` to update its value. 
Otherwise, any predicate with a null dict will set it to false even if previous 
predicates have marked it to true.
   
   Additionally, we may have a chance to shortcut the evaluation as well if any 
predicate has set it to true.



##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/RowGroupFilter.java:
##
@@ -98,16 +99,19 @@ public List 
visit(FilterCompat.FilterPredicateCompat filterPredic
 
 for (BlockMetaData block : blocks) {
   boolean drop = false;
+  // Whether one filter can exactly determine the existence/nonexistence 
of the value.
+  // If true then we can skip the remaining filters to save time and space.
+  AtomicBoolean canExactlyDetermine = new AtomicBoolean(false);

Review Comment:
   I'd suggest rename `canExactlyDetermine` to `preciselyDetermined`. Or even 
better, use an enum something like below
   ```java
   enum PredicateEvaluation {
 CAN_DROP, /* the block can be dropped for sure */
 CANNOT_DROP,  /* the block cannot be dropped for sure*/
 MAY_DROP, /* cannot decide yet, may be dropped by other filter levels 
*/
   }
   ```
   In this way, we can merge the the two boolean values here. The downside is 
that the code may need more refactoring to add the enum value to different 
filter classes.





> 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
>
> Bloomfilter needs to load from filesystem, it may costs time and space. If we 
> can  exactly determine the existence/nonexistence of the value from other 
> filters , then we can avoid using Bloomfilter to Improve performance.
>  
> When the minMax values in  StatisticsFilter is same, we can exactly determine 
> the existence/nonexistence of the value.
> When we have page dictionaries, we can also determine the 
> existence/nonexistence of the value.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

2023-02-04 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

yabola opened a new pull request, #1023:
URL: https://github.com/apache/parquet-mr/pull/1023

   Bloomfilter needs to load from filesystem, it may costs time and space. If 
we can  exactly determine the existence/nonexistence of the value from other 
filters , then we can avoid using Bloomfilter to Improve performance.

   When the minMax values in  StatisticsFilter is same, we can exactly 
determine the existence/nonexistence of the value.
   When we have page dictionaries, we can also determine the 
existence/nonexistence of the value.




> 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
>
> Bloomfilter needs to load from filesystem, it may costs time and space. If we 
> can  exactly determine the existence/nonexistence of the value from other 
> filters , then we can avoid using Bloomfilter to Improve performance.
>  
> When the minMax values in  StatisticsFilter is same, we can exactly determine 
> the existence/nonexistence of the value.
> When we have page dictionaries, we can also determine the 
> existence/nonexistence of the value.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)