You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@drill.apache.org by GitBox <gi...@apache.org> on 2018/07/03 16:18:23 UTC

[GitHub] vrozov commented on a change in pull request #1298: DRILL-5796: Filter pruning for multi rowgroup parquet file

vrozov commented on a change in pull request #1298: DRILL-5796: Filter pruning for multi rowgroup parquet file
URL: https://github.com/apache/drill/pull/1298#discussion_r199868514
 
 

 ##########
 File path: exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/ParquetIsPredicate.java
 ##########
 @@ -62,90 +60,89 @@ private ParquetIsPredicate(LogicalExpression expr, BiPredicate<Statistics<C>, Ra
     return visitor.visitUnknown(this, value);
   }
 
-  @Override
-  public boolean canDrop(RangeExprEvaluator<C> evaluator) {
+  /**
+   * Apply the filter condition against the meta of the rowgroup.
+   */
+  public RowsMatch matches(RangeExprEvaluator<C> evaluator) {
     Statistics<C> exprStat = expr.accept(evaluator, null);
-    if (isNullOrEmpty(exprStat)) {
-      return false;
-    }
+    return ParquetPredicatesHelper.isNullOrEmpty(exprStat) ? RowsMatch.SOME : predicate.apply(exprStat, evaluator);
+  }
 
-    return predicate.test(exprStat, evaluator);
+  /**
+   * After the applying of the filter against the statistics of the rowgroup, if the result is RowsMatch.ALL,
+   * then we still must know if the rowgroup contains some null values, because it can change the filter result.
+   * If it contains some null values, then we change the RowsMatch.ALL into RowsMatch.SOME, which sya that maybe
+   * some values (the null ones) should be disgarded.
+   */
+  static RowsMatch checkNull(Statistics exprStat) {
+    return exprStat.getNumNulls() > 0 ? RowsMatch.SOME : RowsMatch.ALL;
   }
 
   /**
    * IS NULL predicate.
    */
   private static <C extends Comparable<C>> LogicalExpression createIsNullPredicate(LogicalExpression expr) {
     return new ParquetIsPredicate<C>(expr,
-        //if there are no nulls  -> canDrop
-        (exprStat, evaluator) -> hasNoNulls(exprStat)) {
-      private final boolean isArray = isArray(expr);
-
-      private boolean isArray(LogicalExpression expression) {
-        if (expression instanceof TypedFieldExpr) {
-          TypedFieldExpr typedFieldExpr = (TypedFieldExpr) expression;
-          SchemaPath schemaPath = typedFieldExpr.getPath();
-          return schemaPath.isArray();
-        }
-        return false;
-      }
-
-      @Override
-      public boolean canDrop(RangeExprEvaluator<C> evaluator) {
+      (exprStat, evaluator) -> {
         // for arrays we are not able to define exact number of nulls
         // [1,2,3] vs [1,2] -> in second case 3 is absent and thus it's null but statistics shows no nulls
-        return !isArray && super.canDrop(evaluator);
-      }
-    };
+        TypedFieldExpr typedFieldExpr = (TypedFieldExpr) expr;
+        if (typedFieldExpr.getPath().isArray()) {
+          return RowsMatch.SOME;
+        }
+        if (hasNoNulls(exprStat)) {
+          return RowsMatch.NONE;
+        }
+        return isAllNulls(exprStat, evaluator.getRowCount()) ? RowsMatch.ALL : RowsMatch.SOME;
+      });
   }
 
   /**
    * IS NOT NULL predicate.
    */
   private static <C extends Comparable<C>> LogicalExpression createIsNotNullPredicate(LogicalExpression expr) {
     return new ParquetIsPredicate<C>(expr,
-        //if there are all nulls  -> canDrop
-        (exprStat, evaluator) -> isAllNulls(exprStat, evaluator.getRowCount())
+      (exprStat, evaluator) -> isAllNulls(exprStat, evaluator.getRowCount()) ? RowsMatch.NONE : checkNull(exprStat)
     );
   }
 
   /**
    * IS TRUE predicate.
    */
-  private static LogicalExpression createIsTruePredicate(LogicalExpression expr) {
-    return new ParquetIsPredicate<Boolean>(expr,
-        //if max value is not true or if there are all nulls  -> canDrop
-        (exprStat, evaluator) -> !((BooleanStatistics)exprStat).getMax() || isAllNulls(exprStat, evaluator.getRowCount())
-    );
+  private static <C extends Comparable<C>> LogicalExpression createIsTruePredicate(LogicalExpression expr) {
+    return new ParquetIsPredicate<C>(expr,
+      (exprStat, evaluator) -> {
+        if (isAllNulls(exprStat, evaluator.getRowCount()) || (exprStat.genericGetMin().equals(Boolean.FALSE) && exprStat.genericGetMax().equals(Boolean.FALSE))) {
+          return RowsMatch.NONE;
+        }
+        return exprStat.genericGetMin().equals(Boolean.TRUE) && exprStat.genericGetMax().equals(Boolean.TRUE) ? checkNull(exprStat) : RowsMatch.SOME;
+      });
   }
 
   /**
    * IS FALSE predicate.
    */
-  private static LogicalExpression createIsFalsePredicate(LogicalExpression expr) {
-    return new ParquetIsPredicate<Boolean>(expr,
-        //if min value is not false or if there are all nulls  -> canDrop
-        (exprStat, evaluator) -> ((BooleanStatistics)exprStat).getMin() || isAllNulls(exprStat, evaluator.getRowCount())
+  private static <C extends Comparable<C>> LogicalExpression createIsFalsePredicate(LogicalExpression expr) {
+    return new ParquetIsPredicate<C>(expr,
+      (exprStat, evaluator) -> !exprStat.genericGetMin().equals(Boolean.FALSE) || isAllNulls(exprStat, evaluator.getRowCount()) ? RowsMatch.NONE : checkNull(exprStat)
     );
   }
 
   /**
    * IS NOT TRUE predicate.
    */
-  private static LogicalExpression createIsNotTruePredicate(LogicalExpression expr) {
-    return new ParquetIsPredicate<Boolean>(expr,
-        //if min value is not false or if there are no nulls  -> canDrop
-        (exprStat, evaluator) -> ((BooleanStatistics)exprStat).getMin() && hasNoNulls(exprStat)
+  private static <C extends Comparable<C>> LogicalExpression createIsNotTruePredicate(LogicalExpression expr) {
+    return new ParquetIsPredicate<C>(expr,
+      (exprStat, evaluator) -> !exprStat.genericGetMin().equals(Boolean.FALSE) && hasNoNulls(exprStat) ? RowsMatch.NONE : checkNull(exprStat)
     );
   }
 
   /**
    * IS NOT FALSE predicate.
    */
-  private static LogicalExpression createIsNotFalsePredicate(LogicalExpression expr) {
-    return new ParquetIsPredicate<Boolean>(expr,
-        //if max value is not true or if there are no nulls  -> canDrop
-        (exprStat, evaluator) -> !((BooleanStatistics)exprStat).getMax() && hasNoNulls(exprStat)
+  private static <C extends Comparable<C>> LogicalExpression createIsNotFalsePredicate(LogicalExpression expr) {
+    return new ParquetIsPredicate<C>(expr,
+      (exprStat, evaluator) -> !exprStat.genericGetMax().equals(Boolean.TRUE) && hasNoNulls(exprStat) ? RowsMatch.NONE : checkNull(exprStat)
 
 Review comment:
   "is true" can only be applied to a boolean expression and boolean expression can only produce `BooleanStatistics`. It is invalid anyway to compare `Integer`, `Long` or `Double` to `True` or `False`.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services