You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by pa...@apache.org on 2016/11/05 00:11:33 UTC

[06/10] drill git commit: DRILL-1950: Initial prototype patch for parquet filter pushdown.

DRILL-1950: Initial prototype patch for parquet filter pushdown.

Use three new classes from Adam's patch.


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/8ce1f033
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/8ce1f033
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/8ce1f033

Branch: refs/heads/master
Commit: 8ce1f03331970172dfd9d967504c60fd9752716a
Parents: a459e4d
Author: AdamPD <ad...@pharmadata.net.au>
Authored: Thu Jun 30 14:09:53 2016 -0700
Committer: Parth Chandra <pa...@apache.org>
Committed: Fri Nov 4 15:59:31 2016 -0700

----------------------------------------------------------------------
 .../ParquetCompareFunctionProcessor.java        | 280 ++++++++++++++++++
 .../store/parquet/ParquetFilterBuilder.java     | 284 +++++++++++++++++++
 .../store/parquet/ParquetPushDownFilter.java    | 142 ++++++++++
 3 files changed, 706 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/8ce1f033/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetCompareFunctionProcessor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetCompareFunctionProcessor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetCompareFunctionProcessor.java
new file mode 100644
index 0000000..bd59021
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetCompareFunctionProcessor.java
@@ -0,0 +1,280 @@
+/**
+ * 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.drill.exec.store.parquet;
+
+import org.apache.drill.common.expression.CastExpression;
+import org.apache.drill.common.expression.ConvertExpression;
+import org.apache.drill.common.expression.FunctionCall;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.expression.ValueExpressions.BooleanExpression;
+import org.apache.drill.common.expression.ValueExpressions.DateExpression;
+import org.apache.drill.common.expression.ValueExpressions.DoubleExpression;
+import org.apache.drill.common.expression.ValueExpressions.FloatExpression;
+import org.apache.drill.common.expression.ValueExpressions.IntExpression;
+import org.apache.drill.common.expression.ValueExpressions.LongExpression;
+import org.apache.drill.common.expression.ValueExpressions.QuotedString;
+import org.apache.drill.common.expression.ValueExpressions.TimeExpression;
+import org.apache.drill.common.expression.ValueExpressions.TimeStampExpression;
+import org.apache.drill.common.expression.visitors.AbstractExprVisitor;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import org.joda.time.DateTimeUtils;
+
+public class ParquetCompareFunctionProcessor extends
+        AbstractExprVisitor<Boolean, LogicalExpression, RuntimeException> {
+    private Object value;
+    private boolean success;
+    private boolean isEqualityFn;
+    private SchemaPath path;
+    private String functionName;
+
+    public static final long JULIAN_DAY_EPOC = DateTimeUtils.toJulianDayNumber(0);
+
+    public static boolean isCompareFunction(String functionName) {
+        return COMPARE_FUNCTIONS_TRANSPOSE_MAP.keySet().contains(functionName);
+    }
+
+    public static ParquetCompareFunctionProcessor process(FunctionCall call) {
+        String functionName = call.getName();
+        LogicalExpression nameArg = call.args.get(0);
+        LogicalExpression valueArg = call.args.size() == 2 ? call.args.get(1)
+                : null;
+        ParquetCompareFunctionProcessor evaluator = new ParquetCompareFunctionProcessor(
+                functionName);
+
+        if (valueArg != null) { // binary function
+            if (VALUE_EXPRESSION_CLASSES.contains(nameArg.getClass())) {
+                LogicalExpression swapArg = valueArg;
+                valueArg = nameArg;
+                nameArg = swapArg;
+                evaluator.functionName = COMPARE_FUNCTIONS_TRANSPOSE_MAP
+                        .get(functionName);
+            }
+            evaluator.success = nameArg.accept(evaluator, valueArg);
+        } else if (call.args.get(0) instanceof SchemaPath) {
+            evaluator.success = true;
+            evaluator.path = (SchemaPath) nameArg;
+        }
+
+        return evaluator;
+    }
+
+    public ParquetCompareFunctionProcessor(String functionName) {
+        this.success = false;
+        this.functionName = functionName;
+        this.isEqualityFn = COMPARE_FUNCTIONS_TRANSPOSE_MAP
+                .containsKey(functionName)
+                && COMPARE_FUNCTIONS_TRANSPOSE_MAP.get(functionName).equals(
+                functionName);
+    }
+
+    public Object getValue() {
+        return value;
+    }
+
+    public boolean isSuccess() {
+        return success;
+    }
+
+    public SchemaPath getPath() {
+        return path;
+    }
+
+    public String getFunctionName() {
+        return functionName;
+    }
+
+    @Override
+    public Boolean visitCastExpression(CastExpression e,
+                                       LogicalExpression valueArg) throws RuntimeException {
+        if (e.getInput() instanceof CastExpression
+                || e.getInput() instanceof SchemaPath) {
+            return e.getInput().accept(this, valueArg);
+        }
+        return false;
+    }
+
+    @Override
+    public Boolean visitConvertExpression(ConvertExpression e,
+                                          LogicalExpression valueArg) throws RuntimeException {
+        if (e.getConvertFunction() == ConvertExpression.CONVERT_FROM
+                && e.getInput() instanceof SchemaPath) {
+            String encodingType = e.getEncodingType();
+            switch (encodingType) {
+                case "INT_BE":
+                case "INT":
+                case "UINT_BE":
+                case "UINT":
+                case "UINT4_BE":
+                case "UINT4":
+                    if (valueArg instanceof IntExpression
+                            && (isEqualityFn || encodingType.startsWith("U"))) {
+                        this.value = ((IntExpression) valueArg).getInt();
+                    }
+                    break;
+                case "BIGINT_BE":
+                case "BIGINT":
+                case "UINT8_BE":
+                case "UINT8":
+                    if (valueArg instanceof LongExpression
+                            && (isEqualityFn || encodingType.startsWith("U"))) {
+                        this.value = ((LongExpression) valueArg).getLong();
+                    }
+                    break;
+                case "FLOAT":
+                    if (valueArg instanceof FloatExpression && isEqualityFn) {
+                        this.value = ((FloatExpression) valueArg).getFloat();
+                    }
+                    break;
+                case "DOUBLE":
+                    if (valueArg instanceof DoubleExpression && isEqualityFn) {
+                        this.value = ((DoubleExpression) valueArg).getDouble();
+                    }
+                    break;
+                case "TIME_EPOCH":
+                case "TIME_EPOCH_BE":
+                    if (valueArg instanceof TimeExpression) {
+                        this.value = ((TimeExpression) valueArg).getTime();
+                    }
+                    break;
+                case "DATE_EPOCH":
+                case "DATE_EPOCH_BE":
+                    if (valueArg instanceof DateExpression) {
+                        long dateInMillis = ((DateExpression) valueArg).getDate();
+                        this.value = (int) (DateTimeUtils.toJulianDayNumber(dateInMillis) + JULIAN_DAY_EPOC);
+                    }
+                    break;
+                case "BOOLEAN_BYTE":
+                    if (valueArg instanceof BooleanExpression) {
+                        this.value = ((BooleanExpression) valueArg).getBoolean();
+                    }
+                    break;
+                case "UTF8":
+                    // let visitSchemaPath() handle this.
+                    return e.getInput().accept(this, valueArg);
+            }
+
+            if (value != null) {
+                this.path = (SchemaPath) e.getInput();
+                return true;
+            }
+        }
+        return false;
+    }
+
+    @Override
+    public Boolean visitUnknown(LogicalExpression e, LogicalExpression valueArg)
+            throws RuntimeException {
+        return false;
+    }
+
+    @Override
+    public Boolean visitSchemaPath(SchemaPath path, LogicalExpression valueArg)
+            throws RuntimeException {
+        if (valueArg instanceof QuotedString) {
+            this.value = ((QuotedString) valueArg).value;
+            this.path = path;
+            return true;
+        }
+
+        if (valueArg instanceof IntExpression) {
+            this.value = ((IntExpression) valueArg).getInt();
+            this.path = path;
+            return true;
+        }
+
+        if (valueArg instanceof LongExpression) {
+            this.value = ((LongExpression) valueArg).getLong();
+            this.path = path;
+            return true;
+        }
+
+        if (valueArg instanceof FloatExpression) {
+            this.value = ((FloatExpression) valueArg).getFloat();
+            this.path = path;
+            return true;
+        }
+
+        if (valueArg instanceof DoubleExpression) {
+            this.value = ((DoubleExpression) valueArg).getDouble();
+            this.path = path;
+            return true;
+        }
+
+        if (valueArg instanceof BooleanExpression) {
+            this.value = ((BooleanExpression) valueArg).getBoolean();
+            this.path = path;
+            return true;
+        }
+
+        if (valueArg instanceof TimeExpression) {
+            this.value = ((TimeExpression) valueArg).getTime();
+            this.path = path;
+            return true;
+        }
+
+        if (valueArg instanceof TimeStampExpression) {
+            this.value = ((TimeStampExpression) valueArg).getTimeStamp();
+            this.path = path;
+            return true;
+        }
+
+        if (valueArg instanceof DateExpression) {
+            long dateInMillis = ((DateExpression) valueArg).getDate();
+            this.value = (int) (DateTimeUtils.toJulianDayNumber(dateInMillis) + JULIAN_DAY_EPOC);
+            this.path = path;
+            return true;
+        }
+
+        return false;
+    }
+
+    private static final ImmutableSet<Class<? extends LogicalExpression>> VALUE_EXPRESSION_CLASSES;
+    static {
+        ImmutableSet.Builder<Class<? extends LogicalExpression>> builder = ImmutableSet
+                .builder();
+        VALUE_EXPRESSION_CLASSES = builder.add(BooleanExpression.class)
+                .add(DateExpression.class).add(DoubleExpression.class)
+                .add(FloatExpression.class).add(IntExpression.class)
+                .add(LongExpression.class).add(QuotedString.class)
+                .add(TimeExpression.class).add(TimeStampExpression.class)
+                .add(DateExpression.class).build();
+    }
+
+    private static final ImmutableMap<String, String> COMPARE_FUNCTIONS_TRANSPOSE_MAP;
+    static {
+        ImmutableMap.Builder<String, String> builder = ImmutableMap.builder();
+        COMPARE_FUNCTIONS_TRANSPOSE_MAP = builder
+                // unary functions
+                .put("isnotnull", "isnotnull")
+                .put("isNotNull", "isNotNull")
+                .put("is not null", "is not null")
+                .put("isnull", "isnull")
+                .put("isNull", "isNull")
+                .put("is null", "is null")
+                        // binary functions
+                .put("equal", "equal").put("not_equal", "not_equal")
+                .put("greater_than_or_equal_to", "less_than_or_equal_to")
+                .put("greater_than", "less_than")
+                .put("less_than_or_equal_to", "greater_than_or_equal_to")
+                .put("less_than", "greater_than").build();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/8ce1f033/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFilterBuilder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFilterBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFilterBuilder.java
new file mode 100644
index 0000000..b613707
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFilterBuilder.java
@@ -0,0 +1,284 @@
+/**
+ * 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.drill.exec.store.parquet;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.drill.common.expression.BooleanOperator;
+import org.apache.drill.common.expression.FunctionCall;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.expression.visitors.AbstractExprVisitor;
+import org.apache.parquet.filter2.predicate.FilterPredicate;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.parquet.filter2.predicate.FilterApi;
+import org.apache.parquet.filter2.predicate.FilterPredicate;
+
+import java.io.IOException;
+import java.util.List;
+
+public class ParquetFilterBuilder extends
+        AbstractExprVisitor<FilterPredicate, Void, RuntimeException> {
+    static final Logger logger = LoggerFactory
+            .getLogger(ParquetFilterBuilder.class);
+    private LogicalExpression le;
+    private boolean allExpressionsConverted = true;
+    private ParquetGroupScan groupScan;
+
+    public ParquetFilterBuilder(ParquetGroupScan groupScan, LogicalExpression conditionExp) {
+        this.le = conditionExp;
+        this.groupScan = groupScan;
+    }
+
+    public ParquetGroupScan parseTree() {
+        FilterPredicate predicate = le.accept(this, null);
+        try {
+            return this.groupScan.clone(predicate);
+        } catch (IOException e) {
+            logger.error("Failed to set Parquet filter", e);
+            return null;
+        }
+    }
+
+    public boolean areAllExpressionsConverted() {
+        return allExpressionsConverted;
+    }
+
+    @Override
+    public FilterPredicate visitUnknown(LogicalExpression e, Void value) throws RuntimeException {
+        allExpressionsConverted = false;
+        return null;
+    }
+
+    @Override
+    public FilterPredicate visitBooleanOperator(BooleanOperator op, Void value) {
+        List<LogicalExpression> args = op.args;
+        FilterPredicate nodePredicate = null;
+        String functionName = op.getName();
+        for (LogicalExpression arg : args) {
+            switch (functionName) {
+                case "booleanAnd":
+                case "booleanOr":
+                    if (nodePredicate == null) {
+                        nodePredicate = arg.accept(this, null);
+                    } else {
+                        FilterPredicate predicate = arg.accept(this, null);
+                        if (predicate != null) {
+                            nodePredicate = mergePredicates(functionName, nodePredicate, predicate);
+                        } else {
+                            // we can't include any part of the OR if any of the predicates cannot be converted
+                            if (functionName == "booleanOr") {
+                                nodePredicate = null;
+                            }
+                            allExpressionsConverted = false;
+                        }
+                    }
+                    break;
+            }
+        }
+        return nodePredicate;
+    }
+
+    private FilterPredicate mergePredicates(String functionName,
+                                            FilterPredicate leftPredicate, FilterPredicate rightPredicate) {
+        if (leftPredicate != null && rightPredicate != null) {
+            if (functionName == "booleanAnd") {
+                return FilterApi.and(leftPredicate, rightPredicate);
+            }
+            else {
+                return FilterApi.or(leftPredicate, rightPredicate);
+            }
+        } else {
+            allExpressionsConverted = false;
+            if ("booleanAnd".equals(functionName)) {
+                return leftPredicate == null ? rightPredicate : leftPredicate;
+            }
+        }
+
+        return null;
+    }
+
+    @Override
+    public FilterPredicate visitFunctionCall(FunctionCall call, Void value) throws RuntimeException {
+        FilterPredicate predicate = null;
+        String functionName = call.getName();
+        ImmutableList<LogicalExpression> args = call.args;
+
+        if (ParquetCompareFunctionProcessor.isCompareFunction(functionName)) {
+            ParquetCompareFunctionProcessor processor = ParquetCompareFunctionProcessor
+                    .process(call);
+            if (processor.isSuccess()) {
+                try {
+                    predicate = createFilterPredicate(processor.getFunctionName(),
+                            processor.getPath(), processor.getValue());
+                } catch (Exception e) {
+                    logger.error("Failed to create Parquet filter", e);
+                }
+            }
+        } else {
+            switch (functionName) {
+                case "booleanAnd":
+                case "booleanOr":
+                    FilterPredicate leftPredicate = args.get(0).accept(this, null);
+                    FilterPredicate rightPredicate = args.get(1).accept(this, null);
+                    predicate = mergePredicates(functionName, leftPredicate, rightPredicate);
+                    break;
+            }
+        }
+
+        if (predicate == null) {
+            allExpressionsConverted = false;
+        }
+
+        return predicate;
+    }
+
+    private FilterPredicate createFilterPredicate(String functionName,
+                                                  SchemaPath field, Object fieldValue) {
+        FilterPredicate filter = null;
+
+        // extract the field name
+        String fieldName = field.getAsUnescapedPath();
+        switch (functionName) {
+            case "equal":
+                if (fieldValue instanceof Long) {
+                    filter = FilterApi.eq(FilterApi.longColumn(fieldName), (Long) fieldValue);
+                }
+                else if (fieldValue instanceof Integer) {
+                    filter = FilterApi.eq(FilterApi.intColumn(fieldName), (Integer) fieldValue);
+                }
+                else if (fieldValue instanceof Float) {
+                    filter = FilterApi.eq(FilterApi.floatColumn(fieldName), (Float) fieldValue);
+                }
+                else if (fieldValue instanceof Double) {
+                    filter = FilterApi.eq(FilterApi.doubleColumn(fieldName), (Double) fieldValue);
+                }
+                else if (fieldValue instanceof Boolean) {
+                    filter = FilterApi.eq(FilterApi.booleanColumn(fieldName), (Boolean) fieldValue);
+                }
+                break;
+            case "not_equal":
+                if (fieldValue instanceof Long) {
+                    filter = FilterApi.notEq(FilterApi.longColumn(fieldName), (Long) fieldValue);
+                }
+                else if (fieldValue instanceof Integer) {
+                    filter = FilterApi.notEq(FilterApi.intColumn(fieldName), (Integer) fieldValue);
+                }
+                else if (fieldValue instanceof Float) {
+                    filter = FilterApi.notEq(FilterApi.floatColumn(fieldName), (Float) fieldValue);
+                }
+                else if (fieldValue instanceof Double) {
+                    filter = FilterApi.notEq(FilterApi.doubleColumn(fieldName), (Double) fieldValue);
+                }
+                else if (fieldValue instanceof Boolean) {
+                    filter = FilterApi.notEq(FilterApi.booleanColumn(fieldName), (Boolean) fieldValue);
+                }
+                break;
+            case "greater_than_or_equal_to":
+                if (fieldValue instanceof Long) {
+                    filter = FilterApi.gtEq(FilterApi.longColumn(fieldName), (Long) fieldValue);
+                }
+                else if (fieldValue instanceof Integer) {
+                    filter = FilterApi.gtEq(FilterApi.intColumn(fieldName), (Integer) fieldValue);
+                }
+                else if (fieldValue instanceof Float) {
+                    filter = FilterApi.gtEq(FilterApi.floatColumn(fieldName), (Float) fieldValue);
+                }
+                else if (fieldValue instanceof Double) {
+                    filter = FilterApi.gtEq(FilterApi.doubleColumn(fieldName), (Double) fieldValue);
+                }
+                break;
+            case "greater_than":
+                if (fieldValue instanceof Long) {
+                    filter = FilterApi.gt(FilterApi.longColumn(fieldName), (Long) fieldValue);
+                }
+                else if (fieldValue instanceof Integer) {
+                    filter = FilterApi.gt(FilterApi.intColumn(fieldName), (Integer) fieldValue);
+                }
+                else if (fieldValue instanceof Float) {
+                    filter = FilterApi.gt(FilterApi.floatColumn(fieldName), (Float) fieldValue);
+                }
+                else if (fieldValue instanceof Double) {
+                    filter = FilterApi.gt(FilterApi.doubleColumn(fieldName), (Double) fieldValue);
+                }
+                break;
+            case "less_than_or_equal_to":
+                if (fieldValue instanceof Long) {
+                    filter = FilterApi.ltEq(FilterApi.longColumn(fieldName), (Long) fieldValue);
+                }
+                else if (fieldValue instanceof Integer) {
+                    filter = FilterApi.ltEq(FilterApi.intColumn(fieldName), (Integer) fieldValue);
+                }
+                else if (fieldValue instanceof Float) {
+                    filter = FilterApi.ltEq(FilterApi.floatColumn(fieldName), (Float) fieldValue);
+                }
+                else if (fieldValue instanceof Double) {
+                    filter = FilterApi.ltEq(FilterApi.doubleColumn(fieldName), (Double) fieldValue);
+                }
+                break;
+            case "less_than":
+                if (fieldValue instanceof Long) {
+                    filter = FilterApi.lt(FilterApi.longColumn(fieldName), (Long) fieldValue);
+                }
+                else if (fieldValue instanceof Integer) {
+                    filter = FilterApi.lt(FilterApi.intColumn(fieldName), (Integer) fieldValue);
+                }
+                else if (fieldValue instanceof Float) {
+                    filter = FilterApi.lt(FilterApi.floatColumn(fieldName), (Float) fieldValue);
+                }
+                else if (fieldValue instanceof Double) {
+                    filter = FilterApi.lt(FilterApi.doubleColumn(fieldName), (Double) fieldValue);
+                }
+                break;
+            case "isnull":
+            case "isNull":
+            case "is null":
+                if (fieldValue instanceof Long) {
+                    filter = FilterApi.eq(FilterApi.longColumn(fieldName), null);
+                }
+                else if (fieldValue instanceof Integer) {
+                    filter = FilterApi.eq(FilterApi.intColumn(fieldName), null);
+                }
+                else if (fieldValue instanceof Float) {
+                    filter = FilterApi.eq(FilterApi.floatColumn(fieldName), null);
+                }
+                else if (fieldValue instanceof Double) {
+                    filter = FilterApi.eq(FilterApi.doubleColumn(fieldName), null);
+                }
+                break;
+            case "isnotnull":
+            case "isNotNull":
+            case "is not null":
+                if (fieldValue instanceof Long) {
+                    filter = FilterApi.notEq(FilterApi.longColumn(fieldName), null);
+                }
+                else if (fieldValue instanceof Integer) {
+                    filter = FilterApi.notEq(FilterApi.intColumn(fieldName), null);
+                }
+                else if (fieldValue instanceof Float) {
+                    filter = FilterApi.notEq(FilterApi.floatColumn(fieldName), null);
+                }
+                else if (fieldValue instanceof Double) {
+                    filter = FilterApi.notEq(FilterApi.doubleColumn(fieldName), null);
+                }
+                break;
+        }
+
+        return filter;
+    }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/8ce1f033/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetPushDownFilter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetPushDownFilter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetPushDownFilter.java
new file mode 100644
index 0000000..10c817b
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetPushDownFilter.java
@@ -0,0 +1,142 @@
+/**
+ * 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.drill.exec.store.parquet;
+
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rex.RexNode;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.ops.OptimizerRulesContext;
+import org.apache.drill.exec.ops.QueryContext;
+import org.apache.drill.exec.planner.logical.DrillOptiq;
+import org.apache.drill.exec.planner.logical.DrillParseContext;
+import org.apache.drill.exec.planner.logical.RelOptHelper;
+import org.apache.drill.exec.planner.physical.FilterPrel;
+import org.apache.drill.exec.planner.physical.PrelUtil;
+import org.apache.drill.exec.planner.physical.ProjectPrel;
+import org.apache.drill.exec.planner.physical.ScanPrel;
+import org.apache.drill.exec.store.StoragePluginOptimizerRule;
+
+import com.google.common.collect.ImmutableList;
+
+public abstract class ParquetPushDownFilter extends StoragePluginOptimizerRule {
+    public static final StoragePluginOptimizerRule getFilterOnProject(final OptimizerRulesContext context){
+        return new ParquetPushDownFilter(
+                RelOptHelper.some(FilterPrel.class, RelOptHelper.some(ProjectPrel.class, RelOptHelper.any(ScanPrel.class))),
+                "ParquetPushDownFilter:Filter_On_Project", context) {
+
+            @Override
+            public boolean matches(RelOptRuleCall call) {
+                if (!enabled) {
+                    return false;
+                }
+                final ScanPrel scan = call.rel(2);
+                if (scan.getGroupScan() instanceof ParquetGroupScan) {
+                    return super.matches(call);
+                }
+                return false;
+            }
+
+            @Override
+            public void onMatch(RelOptRuleCall call) {
+                final FilterPrel filterRel = call.rel(0);
+                final ProjectPrel projectRel = call.rel(1);
+                final ScanPrel scanRel = call.rel(2);
+                doOnMatch(call, filterRel, projectRel, scanRel);
+            };
+        };
+    }
+
+    public static final StoragePluginOptimizerRule getFilterOnScan(final OptimizerRulesContext context){
+        return new ParquetPushDownFilter(
+                RelOptHelper.some(FilterPrel.class, RelOptHelper.any(ScanPrel.class)),
+                "ParquetPushDownFilter:Filter_On_Scan", context) {
+
+            @Override
+            public boolean matches(RelOptRuleCall call) {
+                if (!enabled) {
+                    return false;
+                }
+                final ScanPrel scan = call.rel(1);
+                if (scan.getGroupScan() instanceof ParquetGroupScan) {
+                    return super.matches(call);
+                }
+                return false;
+            }
+
+            @Override
+            public void onMatch(RelOptRuleCall call) {
+                final FilterPrel filterRel = call.rel(0);
+                final ScanPrel scanRel = call.rel(1);
+                doOnMatch(call, filterRel, null, scanRel);
+            }
+        };
+    }
+
+    private final OptimizerRulesContext context;
+    // private final boolean useNewReader;
+    protected final boolean enabled;
+
+    private ParquetPushDownFilter(RelOptRuleOperand operand, String id, OptimizerRulesContext context) {
+        super(operand, id);
+        this.context = context;
+        this.enabled = context.getPlannerSettings().isParquetFilterPushEnabled();
+        // this.useNewReader = context.getPlannerSettings()getOptions().getOption(ExecConstants.PARQUET_NEW_RECORD_READER).bool_val;
+    }
+
+    protected void doOnMatch(RelOptRuleCall call, FilterPrel filter, ProjectPrel project, ScanPrel scan) {
+        ParquetGroupScan groupScan = (ParquetGroupScan) scan.getGroupScan();
+        if (groupScan.getFilter() != null) {
+            return;
+        }
+
+        RexNode condition = null;
+        if(project == null){
+            condition = filter.getCondition();
+        }else{
+            // get the filter as if it were below the projection.
+            condition = RelOptUtil.pushFilterPastProject(filter.getCondition(), project);
+        }
+
+        LogicalExpression conditionExp = DrillOptiq.toDrill(
+                new DrillParseContext(PrelUtil.getPlannerSettings(call.getPlanner())), scan, condition);
+        ParquetFilterBuilder parquetFilterBuilder = new ParquetFilterBuilder(groupScan,
+                conditionExp);
+        ParquetGroupScan newGroupScan = parquetFilterBuilder.parseTree();
+        if (newGroupScan == null) {
+            return; // no filter pushdown so nothing to apply.
+        }
+
+        final ScanPrel newScanPrel = ScanPrel.create(scan, filter.getTraitSet(),
+                newGroupScan, scan.getRowType());
+
+        RelNode inputPrel = newScanPrel;
+
+        if(project != null){
+            inputPrel = project.copy(project.getTraitSet(), ImmutableList.of(inputPrel));
+        }
+
+        // Normally we could eliminate the filter if all expressions were pushed down;
+        // however, the Parquet filter implementation is type specific (whereas Drill is not)
+        final RelNode newFilter = filter.copy(filter.getTraitSet(), ImmutableList.of(inputPrel));
+        call.transformTo(newFilter);
+    }
+}