[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-22 Thread via GitHub


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 . 
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-22 Thread via GitHub


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. 
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-22 Thread via GitHub


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`.
 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-22 Thread via GitHub


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



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-22 Thread via GitHub


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



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-22 Thread via GitHub


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`.
 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-22 Thread via GitHub


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`.
 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-22 Thread via GitHub


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`.
 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-22 Thread via GitHub


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`
 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-22 Thread via GitHub


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`
 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-22 Thread via GitHub


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`
 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-22 Thread via GitHub


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



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-22 Thread via GitHub


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.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-22 Thread via GitHub


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.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-21 Thread via GitHub


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`
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-21 Thread via GitHub


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`
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-21 Thread via GitHub


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`
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-21 Thread via GitHub


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?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-21 Thread via GitHub


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?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-21 Thread via GitHub


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?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-21 Thread via GitHub


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.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-21 Thread via GitHub


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.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-21 Thread via GitHub


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.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-21 Thread via GitHub


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.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-21 Thread via GitHub


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.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-21 Thread via GitHub


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.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-21 Thread via GitHub


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.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-21 Thread via GitHub


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.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-20 Thread via GitHub


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
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-17 Thread via GitHub


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
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-17 Thread via GitHub


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
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-17 Thread via GitHub


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



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-17 Thread via GitHub


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
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-17 Thread via GitHub


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)
   
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-17 Thread via GitHub


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
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-17 Thread via GitHub


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`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-17 Thread via GitHub


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



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-12 Thread via GitHub


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<>();
+existValues.add(Binary.fromString("miller"));
+existValues.add(Binary.fromString("anderson"));
+
+assertCorrectFiltering(eq(binaryColumn("name"), null));
+assertCorrectFiltering(eq(binaryColumn("name"), 
Binary.fromString("miller")));
+

[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-10 Thread via GitHub


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.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-10 Thread via GitHub


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.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-10 Thread via GitHub


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.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-10 Thread via GitHub


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.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-10 Thread via GitHub


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



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-05 Thread via GitHub


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



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-05 Thread via GitHub


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



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-05 Thread via GitHub


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



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-05 Thread via GitHub


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



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org