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

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

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


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

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



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

Review Comment:
   ditto



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

Review Comment:
   ```suggestion
           if (predicate == BLOCK_MUST_MATCH) {
   ```



##########
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##########
@@ -289,8 +320,14 @@ public <T extends Comparable<T>> Boolean visit(Lt<T> lt) {
 
     T value = lt.getValue();
 
-    // drop if value <= min
-    return stats.compareMinToValue(value) >= 0;
+    // we are looking for records where v < someValue
+    if (stats.compareMinToValue(value) >= 0) {
+      // drop if value <= min
+      return BLOCK_CANNOT_MATCH;
+    } else {
+      // if value > min, we must take it
+      return BLOCK_MUST_MATCH;

Review Comment:
   Sometimes the min/max values are actually lower/upper bounds. Does this 
optimization still work for that case?



##########
parquet-hadoop/src/main/java/org/apache/parquet/filter2/dictionarylevel/DictionaryFilter.java:
##########
@@ -392,15 +420,15 @@ public <T extends Comparable<T>> Boolean visit(In<T> in) {
     try {
       Set<T> dictSet = expandDictionary(meta);
       if (dictSet != null) {
-        return drop(dictSet, values);
+        return predicate(dictSet, values);
       }
     } catch (IOException e) {
       LOG.warn("Failed to process dictionary for filter evaluation.", e);
     }
     return BLOCK_MIGHT_MATCH; // cannot drop the row group based on this 
dictionary
   }
 
-  private <T extends Comparable<T>> Boolean drop(Set<T> dictSet, Set<T> 
values) {
+  private <T extends Comparable<T>> Boolean predicate(Set<T> dictSet, Set<T> 
values) {

Review Comment:
   Use a verb instead of noun.



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

Review Comment:
   We'd better explain explicitly that the evaluation decision it whether to 
drop the row group. That's why BLOCK_MIGHT_MATCH is false and the AND 
expression uses `||` in the implementation. This is counter-intuitive at the 
first glance.



##########
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) {
+  public static Boolean evaluate(FilterPredicate pred, 
List<ColumnChunkMetaData> columns, BloomFilterReader bloomFilterReader) {

Review Comment:
   Why use `Boolean` everywhere? It seems an overkill to me.



##########
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<Boolean> predicate) {
+    Boolean left = and.getLeft().accept(predicate);

Review Comment:
   In the current implementation, left and right predicates are always 
evaluated. We can short cut the evaluation if `left == BLOCK_CANNOT_MATCH` and 
skip evaluation of right predicate. It would save some cost if the right expr 
will read dictionary.



##########
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<Boolean> predicate) {
+    Boolean left = and.getLeft().accept(predicate);
+    Boolean right = and.getRight().accept(predicate);
+    if (left == BLOCK_CANNOT_MATCH || right == BLOCK_CANNOT_MATCH) {
+      return BLOCK_CANNOT_MATCH;
+    } else if (left == BLOCK_MUST_MATCH && right == BLOCK_MUST_MATCH) {
+      // if left and right operation must need the block, then we must take 
the block
+      return BLOCK_MUST_MATCH;
+    } else {
+      return BLOCK_MIGHT_MATCH;
+    }
+  }
+
+  public static Boolean evaluateOr(Operators.Or or, 
FilterPredicate.Visitor<Boolean> predicate) {
+    Boolean left = or.getLeft().accept(predicate);
+    Boolean right = or.getRight().accept(predicate);
+    if (left == BLOCK_CANNOT_MATCH && right == BLOCK_CANNOT_MATCH) {
+      return BLOCK_CANNOT_MATCH;
+    } else if (left == BLOCK_MUST_MATCH || right == BLOCK_MUST_MATCH) {
+      // if left or right operation must need the block, then we must take the 
block
+      return BLOCK_MUST_MATCH;
+    } else {
+      return BLOCK_MIGHT_MATCH;
+    }
+  }
+
+  public static Boolean isDeterminedPredicate(Boolean predicate) {

Review Comment:
   Please add some comments to the public methods.



##########
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<Boolean> predicate) {
+    Boolean left = and.getLeft().accept(predicate);
+    Boolean right = and.getRight().accept(predicate);
+    if (left == BLOCK_CANNOT_MATCH || right == BLOCK_CANNOT_MATCH) {
+      return BLOCK_CANNOT_MATCH;
+    } else if (left == BLOCK_MUST_MATCH && right == BLOCK_MUST_MATCH) {
+      // if left and right operation must need the block, then we must take 
the block
+      return BLOCK_MUST_MATCH;
+    } else {
+      return BLOCK_MIGHT_MATCH;
+    }
+  }
+
+  public static Boolean evaluateOr(Operators.Or or, 
FilterPredicate.Visitor<Boolean> predicate) {
+    Boolean left = or.getLeft().accept(predicate);
+    Boolean right = or.getRight().accept(predicate);
+    if (left == BLOCK_CANNOT_MATCH && right == BLOCK_CANNOT_MATCH) {
+      return BLOCK_CANNOT_MATCH;
+    } else if (left == BLOCK_MUST_MATCH || right == BLOCK_MUST_MATCH) {
+      // if left or right operation must need the block, then we must take the 
block
+      return BLOCK_MUST_MATCH;
+    } else {
+      return BLOCK_MIGHT_MATCH;
+    }
+  }
+
+  public static Boolean isDeterminedPredicate(Boolean predicate) {
+    return predicate == BLOCK_MUST_MATCH || predicate == BLOCK_CANNOT_MATCH;
+  }
+
+  public static void checkPredicate(Boolean predicate) {

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



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

Review Comment:
   Please remove this blank line.



##########
parquet-hadoop/src/main/java/org/apache/parquet/filter2/dictionarylevel/DictionaryFilter.java:
##########
@@ -392,15 +420,15 @@ public <T extends Comparable<T>> Boolean visit(In<T> in) {
     try {
       Set<T> dictSet = expandDictionary(meta);
       if (dictSet != null) {
-        return drop(dictSet, values);
+        return predicate(dictSet, values);
       }
     } catch (IOException e) {
       LOG.warn("Failed to process dictionary for filter evaluation.", e);
     }
     return BLOCK_MIGHT_MATCH; // cannot drop the row group based on this 
dictionary
   }
 
-  private <T extends Comparable<T>> Boolean drop(Set<T> dictSet, Set<T> 
values) {
+  private <T extends Comparable<T>> Boolean predicate(Set<T> dictSet, Set<T> 
values) {

Review Comment:
   ```suggestion
     private <T extends Comparable<T>> Boolean evaluate(Set<T> dictSet, Set<T> 
values) {
   ```



##########
parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/RowGroupFilter.java:
##########
@@ -97,21 +100,37 @@ public List<BlockMetaData> 
visit(FilterCompat.FilterPredicateCompat filterPredic
     List<BlockMetaData> filteredBlocks = new ArrayList<BlockMetaData>();
 
     for (BlockMetaData block : blocks) {
-      boolean drop = false;
+      Boolean predicate = BLOCK_MIGHT_MATCH;
+
+      if (levels.contains(FilterLevel.STATISTICS)) {
+
+        predicate = StatisticsFilter.evaluate(filterPredicate, 
block.getColumns());
 
-      if(levels.contains(FilterLevel.STATISTICS)) {
-        drop = StatisticsFilter.canDrop(filterPredicate, block.getColumns());
+        if(predicate == BLOCK_MUST_MATCH) {
+          filteredBlocks.add(block);
+        }
+        if(isDeterminedPredicate(predicate)) {
+          continue;
+        }
       }
 
-      if(!drop && levels.contains(FilterLevel.DICTIONARY)) {
-        drop = DictionaryFilter.canDrop(filterPredicate, block.getColumns(), 
reader.getDictionaryReader(block));
+      if (levels.contains(FilterLevel.DICTIONARY)) {
+

Review Comment:
   ditto



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

Review Comment:
   It will deal with boxing and unboxing automatically.



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

Review Comment:
   Same comment for style of lines below.



##########
parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java:
##########
@@ -129,8 +139,9 @@ public <T extends Comparable<T>> Boolean visit(Eq<T> eq) {
         return BLOCK_MIGHT_MATCH;
       }
       // we are looking for records where v eq(null)
-      // so drop if there are no nulls in this chunk
-      return !hasNulls(meta);
+      // so if there are no nulls in this chunk, we can drop it,
+      // if there has nulls in this chunk, we must take it
+      return !hasNulls(meta) ? BLOCK_CANNOT_MATCH : BLOCK_MUST_MATCH;

Review Comment:
   I suspect that hasNull will always return precise information. When 
null_count is missing, hasNulls also returns true. Replacing `BLOCK_MUST_MATCH` 
with `BLOCK_MIGHT_MATCH` here makes more sense to me.



##########
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<Boolean> predicate) {
+    Boolean left = and.getLeft().accept(predicate);
+    Boolean right = and.getRight().accept(predicate);
+    if (left == BLOCK_CANNOT_MATCH || right == BLOCK_CANNOT_MATCH) {
+      return BLOCK_CANNOT_MATCH;
+    } else if (left == BLOCK_MUST_MATCH && right == BLOCK_MUST_MATCH) {
+      // if left and right operation must need the block, then we must take 
the block
+      return BLOCK_MUST_MATCH;
+    } else {
+      return BLOCK_MIGHT_MATCH;
+    }
+  }
+
+  public static Boolean evaluateOr(Operators.Or or, 
FilterPredicate.Visitor<Boolean> predicate) {
+    Boolean left = or.getLeft().accept(predicate);
+    Boolean right = or.getRight().accept(predicate);
+    if (left == BLOCK_CANNOT_MATCH && right == BLOCK_CANNOT_MATCH) {
+      return BLOCK_CANNOT_MATCH;
+    } else if (left == BLOCK_MUST_MATCH || right == BLOCK_MUST_MATCH) {
+      // if left or right operation must need the block, then we must take the 
block
+      return BLOCK_MUST_MATCH;
+    } else {
+      return BLOCK_MIGHT_MATCH;
+    }
+  }
+
+  public static Boolean isDeterminedPredicate(Boolean predicate) {
+    return predicate == BLOCK_MUST_MATCH || predicate == BLOCK_CANNOT_MATCH;
+  }
+
+  public static void checkPredicate(Boolean predicate) {

Review Comment:
   If `BLOCK_CANNOT_MATCH` is defined as `Boolean` but not as `boolean`, the 
equality check here may fail if the input predicate is created in other places. 
Either replace `==` with `.equals()` or use `boolean` as your choice.





> 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