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

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

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


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

Review Comment:
   Minor: extra blank line.



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

Review Comment:
   Minor: extra blank line.



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

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



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

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



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

Review Comment:
   ```suggestion
         if(dictSet != null) {
           return dictSet.contains(value) ? BLOCK_MUST_MATCH || 
BLOCK_CANNOT_MATCH;
         }
   ```





> Improve performance when filters in RowGroupFilter can match exactly
> --------------------------------------------------------------------
>
>                 Key: PARQUET-2237
>                 URL: https://issues.apache.org/jira/browse/PARQUET-2237
>             Project: Parquet
>          Issue Type: Improvement
>            Reporter: Mars
>            Priority: Major
>
> If we can accurately judge by the minMax status, we don’t need to load the 
> dictionary from filesystem and compare one by one anymore.
> Similarly , Bloomfilter needs to load from filesystem, it may costs time and 
> memory. If we can exactly determine the existence/nonexistence of the value 
> from minMax or dictionary filters , then we can avoid using Bloomfilter to 
> Improve performance.
> For example,
>  # read data greater than {{x1}} in the block, if minMax in status is all 
> greater than {{{}x1{}}}, then we don't need to read dictionary and compare 
> one by one.
>  # If we already have page dictionaries and have compared one by one, we 
> don't need to read BloomFilter and compare.



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

Reply via email to