You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2020/08/12 13:15:08 UTC

[GitHub] [iceberg] cmathiesen opened a new pull request #1326: Hive: Filter pushdown

cmathiesen opened a new pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326


   Hello! This is a follow-up PR to the Hive IF PR's that got merged recently that adds filter pushdown to the `HiveIcebergInputFormat`. I've added a filter factory to convert the Hive filter to an Iceberg `Expression` and then use the `InputFormatConfig` to set the filter expression for `IcebergInputFormat` to apply to the table scan. 
   
   cc: @rdblue @guilload @massdosage @pvary @rdsr @shardulm94 
   
   Thanks :D 
   


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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdsr commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r476269437



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergInputFormat.java
##########
@@ -51,6 +58,17 @@
 
     forwardConfigSettings(job);
 
+    //Convert Hive filter to Iceberg filter
+    String hiveFilter = job.get(TableScanDesc.FILTER_EXPR_CONF_STR);

Review comment:
       That makes sense. Seems like we cannot do this for predicate and projection pushdown in StorageHandler itself.

##########
File path: mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergFilterFactory.java
##########
@@ -0,0 +1,243 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentFactory;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.And;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.expressions.Not;
+import org.apache.iceberg.expressions.Or;
+import org.apache.iceberg.expressions.UnboundPredicate;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestHiveIcebergFilterFactory {
+
+  @Test
+  public void testEqualsOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().equals("salary", PredicateLeaf.Type.LONG, 3000L).end().build();
+
+    UnboundPredicate expected = Expressions.equal("salary", 3000L);
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertPredicatesMatch(expected, actual);
+  }
+
+  @Test
+  public void testNotEqualsOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startNot().equals("salary", PredicateLeaf.Type.LONG, 3000L).end().build();
+
+    Not expected = (Not) Expressions.not(Expressions.equal("salary", 3000L));
+    Not actual = (Not) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    UnboundPredicate childExpressionActual = (UnboundPredicate) actual.child();
+    UnboundPredicate childExpressionExpected = Expressions.equal("salary", 3000L);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.child().op(), expected.child().op());
+    assertEquals(childExpressionActual.ref().name(), childExpressionExpected.ref().name());
+    assertEquals(childExpressionActual.literal(), childExpressionExpected.literal());
+  }
+
+  @Test
+  public void testLessThanOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().lessThan("salary", PredicateLeaf.Type.LONG, 3000L).end().build();
+
+    UnboundPredicate expected = Expressions.lessThan("salary", 3000L);
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.literal(), expected.literal());
+    assertEquals(actual.ref().name(), expected.ref().name());
+  }
+
+  @Test
+  public void testLessThanEqualsOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().lessThanEquals("salary", PredicateLeaf.Type.LONG, 3000L).end().build();
+
+    UnboundPredicate expected = Expressions.lessThanOrEqual("salary", 3000L);
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertPredicatesMatch(expected, actual);
+  }
+
+  @Test
+  public void testInOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().in("salary", PredicateLeaf.Type.LONG, 3000L, 4000L).end().build();
+
+    UnboundPredicate expected = Expressions.in("salary", 3000L, 4000L);
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.literals(), expected.literals());
+    assertEquals(actual.ref().name(), expected.ref().name());
+  }
+
+  @Test
+  public void testBetweenOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder
+        .startAnd()
+        .between("salary", PredicateLeaf.Type.LONG, 3000L, 4000L).end().build();
+
+    And expected = (And) Expressions.and(Expressions.greaterThanOrEqual("salary", 3000L),
+        Expressions.lessThanOrEqual("salary", 3000L));
+    And actual = (And) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.left().op(), expected.left().op());
+    assertEquals(actual.right().op(), expected.right().op());
+  }
+
+  @Test
+  public void testIsNullOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().isNull("salary", PredicateLeaf.Type.LONG).end().build();
+
+    UnboundPredicate expected = Expressions.isNull("salary");
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.ref().name(), expected.ref().name());
+  }
+
+  @Test
+  public void testAndOperand() {

Review comment:
       Do we need a couple tests using HiveRunner ? I have a feeling Hive will lowercase the predicate pushdown and then do we need to support a case insensitive match?
   cc @pvary, @guilload 

##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+
+public class HiveIcebergFilterFactory {
+
+  private HiveIcebergFilterFactory() {}
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(childNodes.get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new UnsupportedOperationException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToLiteral(leaf));
+      case LESS_THAN:
+        return lessThan(column, leafToLiteral(leaf));
+      case LESS_THAN_EQUALS:
+        return lessThanOrEqual(column, leafToLiteral(leaf));
+      case IN:
+        return in(column, leafToLiteralList(leaf));
+      case BETWEEN:
+        List<Object> icebergLiterals = leafToLiteralList(leaf);
+        return and(greaterThanOrEqual(column, icebergLiterals.get(0)),
+                lessThanOrEqual(column, icebergLiterals.get(1)));
+      case IS_NULL:
+        return isNull(column);
+      default:
+        throw new UnsupportedOperationException("Unknown operator: " + leaf.getOperator());
+    }
+  }
+
+  private static Object leafToLiteral(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+      case BOOLEAN:
+      case STRING:
+      case FLOAT:
+        return leaf.getLiteral();
+      case DATE:
+        //Hive converts a Date type to a Timestamp internally when retrieving literal
+        return timestampToDateString((Timestamp) leaf.getLiteral());
+      case DECIMAL:
+        return hiveDecimalToBigDecimal((HiveDecimalWritable) leaf.getLiteral());
+      case TIMESTAMP:
+        return timestampToUnixEpoch((Timestamp) leaf.getLiteral());
+
+      default:
+        throw new UnsupportedOperationException("Unknown type: " + leaf.getType());
+    }
+  }
+
+  private static List<Object> leafToLiteralList(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+      case BOOLEAN:
+      case FLOAT:
+      case STRING:
+        return leaf.getLiteralList();
+      case DATE:
+        return leaf.getLiteralList().stream().map(value -> dateToString((Date) value))
+                .collect(Collectors.toList());
+      case DECIMAL:
+        return leaf.getLiteralList().stream()
+                .map(value -> hiveDecimalToBigDecimal((HiveDecimalWritable) value))
+                .collect(Collectors.toList());
+      case TIMESTAMP:
+        return leaf.getLiteralList().stream()
+                .map(value -> timestampToUnixEpoch((Timestamp) value))
+                .collect(Collectors.toList());
+      default:
+        throw new UnsupportedOperationException("Unknown type: " + leaf.getType());
+    }
+  }
+
+  private static String timestampToDateString(Timestamp timestamp) {
+    return timestamp.toLocalDateTime().toLocalDate().toString();
+  }
+
+  private static String dateToString(Date date) {
+    return date.toLocalDate().toString();
+  }
+
+  private static BigDecimal hiveDecimalToBigDecimal(HiveDecimalWritable hiveDecimalWritable) {
+    return hiveDecimalWritable.getHiveDecimal().bigDecimalValue().setScale(hiveDecimalWritable.scale());
+  }
+
+  private static long timestampToUnixEpoch(Timestamp timestamp) {
+    return timestamp.toInstant().getEpochSecond() * TimeUnit.SECONDS.toMicros(1) +
+            timestamp.getNanos() / TimeUnit.MICROSECONDS.toNanos(1);

Review comment:
       +1, similarly for `timestamp.getNanos() / TimeUnit.MICROSECONDS.toNanos(1)` -`TimeUnit.NANOSECONDS.toMicros(timestamp.getNanos())`




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r469401730



##########
File path: mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergFilterFactory.java
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.
+ */
+

Review comment:
       I agree we should tests as many types as we can. We also don't need to test every predicate for every type. I think it's fine to test each predicate with a long and then to test each type with equals, for example.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] cmathiesen commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
cmathiesen commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r469377728



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+
+public class HiveIcebergFilterFactory {
+
+  private HiveIcebergFilterFactory() {}
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(tree.getChildren().get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        //We are unsure of how the CONSTANT case works, so using the approach of:
+        //https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/
+        // ParquetFilterPredicateConverter.java#L116

Review comment:
       Ah yes, thanks for the spot - I changed the approach here during another review and didn't remove the comment. Will remove :) 




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r474804819



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
##########
@@ -140,18 +137,34 @@ private static Object leafToLiteral(PredicateLeaf leaf) {
       case STRING:
         return leaf.getLiteralList();
       case DATE:
-        return leaf.getLiteralList().stream().map(value -> ((Date) value).toLocalDate().toEpochDay())
+        return leaf.getLiteralList().stream().map(value -> dateToString((Date) value))
                 .collect(Collectors.toList());
       case DECIMAL:
         return leaf.getLiteralList().stream()
-                .map(value -> BigDecimal.valueOf(((HiveDecimalWritable) value).doubleValue()))
+                .map(value -> hiveDecimalToBigDecimal((HiveDecimalWritable) value))
                 .collect(Collectors.toList());
       case TIMESTAMP:
         return leaf.getLiteralList().stream()
-                .map(value -> ((Timestamp) value).toInstant().getEpochSecond() * MICROS_PER_SECOND +
-                        ((Timestamp) value).getNanos() / NANOS_PER_MICROSEC).collect(Collectors.toList());
+                .map(value -> timestampToTimestampString((Timestamp) value))
+                .collect(Collectors.toList());
       default:
-        throw new IllegalStateException("Unknown type: " + leaf.getType());
+        throw new UnsupportedOperationException("Unknown type: " + leaf.getType());
     }
   }
+
+  private static String timestampToDateString(Timestamp timestamp) {
+    return timestamp.toLocalDateTime().toLocalDate().toString();
+  }
+
+  private static String dateToString(Date date) {
+    return date.toLocalDate().toString();
+  }
+
+  private static BigDecimal hiveDecimalToBigDecimal(HiveDecimalWritable hiveDecimalWritable) {
+    return new BigDecimal(hiveDecimalWritable.toString()).setScale(hiveDecimalWritable.scale());

Review comment:
       I don't think we want to convert to String here, either. Can you use the same logic from ORC?




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r475827268



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+
+public class HiveIcebergFilterFactory {
+
+  private HiveIcebergFilterFactory() {}
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(childNodes.get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new UnsupportedOperationException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToLiteral(leaf));
+      case LESS_THAN:
+        return lessThan(column, leafToLiteral(leaf));
+      case LESS_THAN_EQUALS:
+        return lessThanOrEqual(column, leafToLiteral(leaf));
+      case IN:
+        return in(column, leafToLiteralList(leaf));
+      case BETWEEN:
+        List<Object> icebergLiterals = leafToLiteralList(leaf);
+        return and(greaterThanOrEqual(column, icebergLiterals.get(0)),
+                lessThanOrEqual(column, icebergLiterals.get(1)));
+      case IS_NULL:
+        return isNull(column);
+      default:
+        throw new UnsupportedOperationException("Unknown operator: " + leaf.getOperator());
+    }
+  }
+
+  private static Object leafToLiteral(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+      case BOOLEAN:
+      case STRING:
+      case FLOAT:
+        return leaf.getLiteral();
+      case DATE:
+        //Hive converts a Date type to a Timestamp internally when retrieving literal
+        return timestampToDateString((Timestamp) leaf.getLiteral());
+      case DECIMAL:
+        return hiveDecimalToBigDecimal((HiveDecimalWritable) leaf.getLiteral());
+      case TIMESTAMP:
+        return timestampToUnixEpoch((Timestamp) leaf.getLiteral());
+
+      default:
+        throw new UnsupportedOperationException("Unknown type: " + leaf.getType());
+    }
+  }
+
+  private static List<Object> leafToLiteralList(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+      case BOOLEAN:
+      case FLOAT:
+      case STRING:
+        return leaf.getLiteralList();
+      case DATE:
+        return leaf.getLiteralList().stream().map(value -> dateToString((Date) value))
+                .collect(Collectors.toList());
+      case DECIMAL:
+        return leaf.getLiteralList().stream()
+                .map(value -> hiveDecimalToBigDecimal((HiveDecimalWritable) value))
+                .collect(Collectors.toList());
+      case TIMESTAMP:
+        return leaf.getLiteralList().stream()
+                .map(value -> timestampToUnixEpoch((Timestamp) value))
+                .collect(Collectors.toList());
+      default:
+        throw new UnsupportedOperationException("Unknown type: " + leaf.getType());
+    }
+  }
+
+  private static String timestampToDateString(Timestamp timestamp) {
+    return timestamp.toLocalDateTime().toLocalDate().toString();

Review comment:
       Dates need to be converted directly to a value and not a string also. You can use `DateTimeUtil` if you need.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r479703649



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.util.DateTimeUtil;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+
+public class HiveIcebergFilterFactory {
+
+  private HiveIcebergFilterFactory() {
+  }
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(childNodes.get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new UnsupportedOperationException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToLiteral(leaf));
+      case LESS_THAN:
+        return lessThan(column, leafToLiteral(leaf));
+      case LESS_THAN_EQUALS:
+        return lessThanOrEqual(column, leafToLiteral(leaf));
+      case IN:
+        return in(column, leafToLiteralList(leaf));
+      case BETWEEN:
+        List<Object> icebergLiterals = leafToLiteralList(leaf);
+        return and(greaterThanOrEqual(column, icebergLiterals.get(0)),
+                lessThanOrEqual(column, icebergLiterals.get(1)));
+      case IS_NULL:
+        return isNull(column);
+      default:
+        throw new UnsupportedOperationException("Unknown operator: " + leaf.getOperator());
+    }
+  }
+
+  private static Object leafToLiteral(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+      case BOOLEAN:
+      case STRING:
+      case FLOAT:
+        return leaf.getLiteral();
+      case DATE:

Review comment:
       Hive may return a Timestamp for date here, but Iceberg needs to convert to an integer number of days. So this still needs to call `daysFromDate`:
   
   ```java
         case DATE:
           return daysFromDate((Date) leaf.getLiteral());
   ```




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r474804542



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
##########
@@ -140,18 +137,34 @@ private static Object leafToLiteral(PredicateLeaf leaf) {
       case STRING:
         return leaf.getLiteralList();
       case DATE:
-        return leaf.getLiteralList().stream().map(value -> ((Date) value).toLocalDate().toEpochDay())
+        return leaf.getLiteralList().stream().map(value -> dateToString((Date) value))
                 .collect(Collectors.toList());
       case DECIMAL:
         return leaf.getLiteralList().stream()
-                .map(value -> BigDecimal.valueOf(((HiveDecimalWritable) value).doubleValue()))
+                .map(value -> hiveDecimalToBigDecimal((HiveDecimalWritable) value))
                 .collect(Collectors.toList());
       case TIMESTAMP:
         return leaf.getLiteralList().stream()
-                .map(value -> ((Timestamp) value).toInstant().getEpochSecond() * MICROS_PER_SECOND +
-                        ((Timestamp) value).getNanos() / NANOS_PER_MICROSEC).collect(Collectors.toList());
+                .map(value -> timestampToTimestampString((Timestamp) value))

Review comment:
       This shouldn't convert to a string. Instead, it should convert the `Timestamp` value directly to microseconds from the unix epoch. String conversion in expressions is only for convenience in tests and for people using the API directly with generics. If an engine passes a predicate, we don't want to needlessly convert to string and back because it is much, much more likely to corrupt the value.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] massdosage commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
massdosage commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r472929907



##########
File path: api/src/main/java/org/apache/iceberg/ContentFile.java
##########
@@ -102,6 +102,18 @@
    */
   List<Long> splitOffsets();
 
+  /**

Review comment:
       @cmathiesen I think something went wrong with your rebase/merge from master, there's a lot of stuff in here now which shouldn't be.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] cmathiesen commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
cmathiesen commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r469378198



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+
+public class HiveIcebergFilterFactory {
+
+  private HiveIcebergFilterFactory() {}
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(tree.getChildren().get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        //We are unsure of how the CONSTANT case works, so using the approach of:
+        //https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/
+        // ParquetFilterPredicateConverter.java#L116
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new IllegalStateException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToIcebergType(leaf));
+      case LESS_THAN:
+        return lessThan(column, leafToIcebergType(leaf));
+      case LESS_THAN_EQUALS:
+        return lessThanOrEqual(column, leafToIcebergType(leaf));
+      case IN:
+        return in(column, (List) leafToIcebergType(leaf));
+      case BETWEEN:
+        List<Object> icebergLiterals = leaf.getLiteralList();
+        return and(greaterThanOrEqual(column, icebergLiterals.get(0)),
+                lessThanOrEqual(column, icebergLiterals.get(1)));
+      case IS_NULL:
+        return isNull(column);
+      default:
+        throw new IllegalStateException("Unknown operator: " + leaf.getOperator());
+    }
+  }
+
+  private static Object leafToIcebergType(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+        return leaf.getLiteral() != null ? leaf.getLiteral() : leaf.getLiteralList();
+      case FLOAT:
+        return leaf.getLiteral() != null ? leaf.getLiteral() : leaf.getLiteralList();
+      case STRING:
+        return leaf.getLiteral() != null ? leaf.getLiteral() : leaf.getLiteralList();
+      case DATE:
+        //Hive converts a Date type to a Timestamp internally when retrieving literal
+        if (leaf.getLiteral() != null) {
+          return ((Timestamp) leaf.getLiteral()).toLocalDateTime().toLocalDate().toEpochDay();
+        } else {
+          //But not when retrieving the literalList
+          List<Object> icebergValues = leaf.getLiteralList();
+          icebergValues.replaceAll(value -> ((Date) value).toLocalDate().toEpochDay());
+          return icebergValues;
+        }
+      case DECIMAL:
+        if (leaf.getLiteral() != null) {
+          return BigDecimal.valueOf(((HiveDecimalWritable) leaf.getLiteral()).doubleValue());
+        } else {
+          List<Object> icebergValues = leaf.getLiteralList();
+          icebergValues.replaceAll(value -> BigDecimal.valueOf(((HiveDecimalWritable) value).doubleValue()));
+          return icebergValues;
+        }
+      case TIMESTAMP:
+        if (leaf.getLiteral() != null) {
+          Timestamp timestamp = (Timestamp) leaf.getLiteral();
+          return timestamp.toInstant().getEpochSecond() * 1000000 + timestamp.getNanos() / 1000;

Review comment:
       A very good point, I'll update these!




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r479703594



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.util.DateTimeUtil;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+
+public class HiveIcebergFilterFactory {
+
+  private HiveIcebergFilterFactory() {
+  }
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(childNodes.get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new UnsupportedOperationException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToLiteral(leaf));
+      case LESS_THAN:
+        return lessThan(column, leafToLiteral(leaf));
+      case LESS_THAN_EQUALS:
+        return lessThanOrEqual(column, leafToLiteral(leaf));
+      case IN:
+        return in(column, leafToLiteralList(leaf));
+      case BETWEEN:
+        List<Object> icebergLiterals = leafToLiteralList(leaf);
+        return and(greaterThanOrEqual(column, icebergLiterals.get(0)),
+                lessThanOrEqual(column, icebergLiterals.get(1)));
+      case IS_NULL:
+        return isNull(column);
+      default:
+        throw new UnsupportedOperationException("Unknown operator: " + leaf.getOperator());
+    }
+  }
+
+  private static Object leafToLiteral(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+      case BOOLEAN:
+      case STRING:
+      case FLOAT:
+        return leaf.getLiteral();
+      case DATE:
+      case TIMESTAMP:
+        // Hive converts a Date type to a Timestamp internally when retrieving literal
+        return timestampToUnixEpoch((Timestamp) leaf.getLiteral());
+      case DECIMAL:
+        return hiveDecimalToBigDecimal((HiveDecimalWritable) leaf.getLiteral());
+
+      default:
+        throw new UnsupportedOperationException("Unknown type: " + leaf.getType());
+    }
+  }
+
+  private static List<Object> leafToLiteralList(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+      case BOOLEAN:
+      case FLOAT:
+      case STRING:
+        return leaf.getLiteralList();
+      case DATE:
+        return leaf.getLiteralList().stream().map(value -> dateToMicros((Date) value))
+                .collect(Collectors.toList());
+      case DECIMAL:
+        return leaf.getLiteralList().stream()
+                .map(value -> hiveDecimalToBigDecimal((HiveDecimalWritable) value))
+                .collect(Collectors.toList());
+      case TIMESTAMP:
+        return leaf.getLiteralList().stream()
+                .map(value -> timestampToUnixEpoch((Timestamp) value))
+                .collect(Collectors.toList());
+      default:
+        throw new UnsupportedOperationException("Unknown type: " + leaf.getType());
+    }
+  }
+
+  private static long dateToMicros(Date date) {
+    return TimeUnit.MILLISECONDS.toMicros(date.toInstant().toEpochMilli());
+  }
+
+  private static BigDecimal hiveDecimalToBigDecimal(HiveDecimalWritable hiveDecimalWritable) {
+    return hiveDecimalWritable.getHiveDecimal().bigDecimalValue().setScale(hiveDecimalWritable.scale());
+  }
+
+  private static long timestampToUnixEpoch(Timestamp timestamp) {

Review comment:
       How about renaming this to `microsFromTimestamp`?
   
   ```java
     private static long microsFromTimestamp(Timestamp timestamp) {
       return DateTimeUtil.microsFromTimestamp(timestamp.toLocalDateTime());
     }
   ```

##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.util.DateTimeUtil;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+
+public class HiveIcebergFilterFactory {
+
+  private HiveIcebergFilterFactory() {
+  }
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(childNodes.get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new UnsupportedOperationException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToLiteral(leaf));
+      case LESS_THAN:
+        return lessThan(column, leafToLiteral(leaf));
+      case LESS_THAN_EQUALS:
+        return lessThanOrEqual(column, leafToLiteral(leaf));
+      case IN:
+        return in(column, leafToLiteralList(leaf));
+      case BETWEEN:
+        List<Object> icebergLiterals = leafToLiteralList(leaf);
+        return and(greaterThanOrEqual(column, icebergLiterals.get(0)),
+                lessThanOrEqual(column, icebergLiterals.get(1)));
+      case IS_NULL:
+        return isNull(column);
+      default:
+        throw new UnsupportedOperationException("Unknown operator: " + leaf.getOperator());
+    }
+  }
+
+  private static Object leafToLiteral(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+      case BOOLEAN:
+      case STRING:
+      case FLOAT:
+        return leaf.getLiteral();
+      case DATE:
+      case TIMESTAMP:
+        // Hive converts a Date type to a Timestamp internally when retrieving literal
+        return timestampToUnixEpoch((Timestamp) leaf.getLiteral());
+      case DECIMAL:
+        return hiveDecimalToBigDecimal((HiveDecimalWritable) leaf.getLiteral());
+
+      default:
+        throw new UnsupportedOperationException("Unknown type: " + leaf.getType());
+    }
+  }
+
+  private static List<Object> leafToLiteralList(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+      case BOOLEAN:
+      case FLOAT:
+      case STRING:
+        return leaf.getLiteralList();
+      case DATE:
+        return leaf.getLiteralList().stream().map(value -> dateToMicros((Date) value))
+                .collect(Collectors.toList());
+      case DECIMAL:
+        return leaf.getLiteralList().stream()
+                .map(value -> hiveDecimalToBigDecimal((HiveDecimalWritable) value))
+                .collect(Collectors.toList());
+      case TIMESTAMP:
+        return leaf.getLiteralList().stream()
+                .map(value -> timestampToUnixEpoch((Timestamp) value))
+                .collect(Collectors.toList());
+      default:
+        throw new UnsupportedOperationException("Unknown type: " + leaf.getType());
+    }
+  }
+
+  private static long dateToMicros(Date date) {

Review comment:
       Dates in Iceberg are the number of days from the unix epoch as an integer, not microseconds. I think this method should be:
   
   ```java
     private static long daysFromDate(Date date) {
       return DateTimeUtil.daysFromDate(date.toLocalDate());
     }
   ```

##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.util.DateTimeUtil;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+
+public class HiveIcebergFilterFactory {
+
+  private HiveIcebergFilterFactory() {
+  }
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(childNodes.get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new UnsupportedOperationException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToLiteral(leaf));
+      case LESS_THAN:
+        return lessThan(column, leafToLiteral(leaf));
+      case LESS_THAN_EQUALS:
+        return lessThanOrEqual(column, leafToLiteral(leaf));
+      case IN:
+        return in(column, leafToLiteralList(leaf));
+      case BETWEEN:
+        List<Object> icebergLiterals = leafToLiteralList(leaf);
+        return and(greaterThanOrEqual(column, icebergLiterals.get(0)),
+                lessThanOrEqual(column, icebergLiterals.get(1)));
+      case IS_NULL:
+        return isNull(column);
+      default:
+        throw new UnsupportedOperationException("Unknown operator: " + leaf.getOperator());
+    }
+  }
+
+  private static Object leafToLiteral(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+      case BOOLEAN:
+      case STRING:
+      case FLOAT:
+        return leaf.getLiteral();
+      case DATE:
+      case TIMESTAMP:
+        // Hive converts a Date type to a Timestamp internally when retrieving literal
+        return timestampToUnixEpoch((Timestamp) leaf.getLiteral());
+      case DECIMAL:
+        return hiveDecimalToBigDecimal((HiveDecimalWritable) leaf.getLiteral());
+
+      default:
+        throw new UnsupportedOperationException("Unknown type: " + leaf.getType());
+    }
+  }
+
+  private static List<Object> leafToLiteralList(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+      case BOOLEAN:
+      case FLOAT:
+      case STRING:
+        return leaf.getLiteralList();
+      case DATE:
+        return leaf.getLiteralList().stream().map(value -> dateToMicros((Date) value))

Review comment:
       Iceberg uses days for a date, not micros, so this should call `daysFromDate`.

##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.util.DateTimeUtil;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+
+public class HiveIcebergFilterFactory {
+
+  private HiveIcebergFilterFactory() {
+  }
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(childNodes.get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new UnsupportedOperationException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToLiteral(leaf));
+      case LESS_THAN:
+        return lessThan(column, leafToLiteral(leaf));
+      case LESS_THAN_EQUALS:
+        return lessThanOrEqual(column, leafToLiteral(leaf));
+      case IN:
+        return in(column, leafToLiteralList(leaf));
+      case BETWEEN:
+        List<Object> icebergLiterals = leafToLiteralList(leaf);
+        return and(greaterThanOrEqual(column, icebergLiterals.get(0)),
+                lessThanOrEqual(column, icebergLiterals.get(1)));
+      case IS_NULL:
+        return isNull(column);
+      default:
+        throw new UnsupportedOperationException("Unknown operator: " + leaf.getOperator());
+    }
+  }
+
+  private static Object leafToLiteral(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+      case BOOLEAN:
+      case STRING:
+      case FLOAT:
+        return leaf.getLiteral();
+      case DATE:

Review comment:
       Hive may return a Timestamp for date here, but Iceberg needs to convert to an integer number of days. So this still needs to call `daysFromDate`:
   
   ```java
         case DATE:
           return daysFromDate((Date) leaf.getLiteral());
   ```

##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.util.DateTimeUtil;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+
+public class HiveIcebergFilterFactory {
+
+  private HiveIcebergFilterFactory() {
+  }
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(childNodes.get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new UnsupportedOperationException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToLiteral(leaf));
+      case LESS_THAN:
+        return lessThan(column, leafToLiteral(leaf));
+      case LESS_THAN_EQUALS:
+        return lessThanOrEqual(column, leafToLiteral(leaf));
+      case IN:
+        return in(column, leafToLiteralList(leaf));
+      case BETWEEN:
+        List<Object> icebergLiterals = leafToLiteralList(leaf);
+        return and(greaterThanOrEqual(column, icebergLiterals.get(0)),
+                lessThanOrEqual(column, icebergLiterals.get(1)));
+      case IS_NULL:
+        return isNull(column);
+      default:
+        throw new UnsupportedOperationException("Unknown operator: " + leaf.getOperator());
+    }
+  }
+
+  private static Object leafToLiteral(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+      case BOOLEAN:
+      case STRING:
+      case FLOAT:
+        return leaf.getLiteral();
+      case DATE:
+      case TIMESTAMP:
+        // Hive converts a Date type to a Timestamp internally when retrieving literal

Review comment:
       This comment isn't needed.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdsr commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r472531652



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
##########
@@ -0,0 +1,159 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+
+public class HiveIcebergFilterFactory {
+
+  private static final int MICROS_PER_SECOND = 1000000;
+  private static final int NANOSECS_PER_MICROSEC = 1000;
+
+  private HiveIcebergFilterFactory() {}
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(childNodes.get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new IllegalStateException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToLiteral(leaf));
+      case LESS_THAN:
+        return lessThan(column, leafToLiteral(leaf));
+      case LESS_THAN_EQUALS:
+        return lessThanOrEqual(column, leafToLiteral(leaf));
+      case IN:
+        return in(column, leafToLiteralList(leaf));
+      case BETWEEN:
+        List<Object> icebergLiterals = leafToLiteralList(leaf);
+        return and(greaterThanOrEqual(column, icebergLiterals.get(0)),
+                lessThanOrEqual(column, icebergLiterals.get(1)));
+      case IS_NULL:
+        return isNull(column);
+      default:
+        throw new IllegalStateException("Unknown operator: " + leaf.getOperator());
+    }
+  }
+
+  private static Object leafToLiteral(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+      case BOOLEAN:
+      case STRING:
+      case FLOAT:
+        return leaf.getLiteral();
+      case DATE:
+        //Hive converts a Date type to a Timestamp internally when retrieving literal
+        return ((Timestamp) leaf.getLiteral()).toLocalDateTime().toLocalDate().toEpochDay();
+      case DECIMAL:
+        return BigDecimal.valueOf(((HiveDecimalWritable) leaf.getLiteral()).doubleValue());
+      case TIMESTAMP:
+        Timestamp timestamp = (Timestamp) leaf.getLiteral();
+        return timestamp.toInstant().getEpochSecond() * MICROS_PER_SECOND +
+                timestamp.getNanos() / NANOSECS_PER_MICROSEC;
+      default:
+        throw new IllegalStateException("Unknown type: " + leaf.getType());
+    }
+  }
+
+  private static List<Object> leafToLiteralList(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+      case BOOLEAN:
+      case FLOAT:
+      case STRING:
+        return leaf.getLiteralList();
+      case DATE:
+        List<Object> dateValues = leaf.getLiteralList();
+        dateValues.replaceAll(value -> ((Date) value).toLocalDate().toEpochDay());
+        return dateValues;
+      case DECIMAL:
+        List<Object> decimalValues = leaf.getLiteralList();

Review comment:
       nit: I think it is clearer to not modify the returned list but to use standard idioms like `leaf.getLiteralList().stream().map` ... or `Lists.transform(..)`

##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
##########
@@ -0,0 +1,159 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+
+public class HiveIcebergFilterFactory {
+
+  private static final int MICROS_PER_SECOND = 1000000;
+  private static final int NANOSECS_PER_MICROSEC = 1000;
+
+  private HiveIcebergFilterFactory() {}
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(childNodes.get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new IllegalStateException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToLiteral(leaf));
+      case LESS_THAN:
+        return lessThan(column, leafToLiteral(leaf));
+      case LESS_THAN_EQUALS:
+        return lessThanOrEqual(column, leafToLiteral(leaf));
+      case IN:
+        return in(column, leafToLiteralList(leaf));
+      case BETWEEN:
+        List<Object> icebergLiterals = leafToLiteralList(leaf);
+        return and(greaterThanOrEqual(column, icebergLiterals.get(0)),
+                lessThanOrEqual(column, icebergLiterals.get(1)));
+      case IS_NULL:
+        return isNull(column);
+      default:
+        throw new IllegalStateException("Unknown operator: " + leaf.getOperator());
+    }
+  }
+
+  private static Object leafToLiteral(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+      case BOOLEAN:
+      case STRING:
+      case FLOAT:
+        return leaf.getLiteral();
+      case DATE:
+        //Hive converts a Date type to a Timestamp internally when retrieving literal
+        return ((Timestamp) leaf.getLiteral()).toLocalDateTime().toLocalDate().toEpochDay();
+      case DECIMAL:
+        return BigDecimal.valueOf(((HiveDecimalWritable) leaf.getLiteral()).doubleValue());
+      case TIMESTAMP:
+        Timestamp timestamp = (Timestamp) leaf.getLiteral();
+        return timestamp.toInstant().getEpochSecond() * MICROS_PER_SECOND +
+                timestamp.getNanos() / NANOSECS_PER_MICROSEC;
+      default:
+        throw new IllegalStateException("Unknown type: " + leaf.getType());
+    }
+  }
+
+  private static List<Object> leafToLiteralList(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+      case BOOLEAN:
+      case FLOAT:
+      case STRING:
+        return leaf.getLiteralList();
+      case DATE:
+        List<Object> dateValues = leaf.getLiteralList();
+        dateValues.replaceAll(value -> ((Date) value).toLocalDate().toEpochDay());
+        return dateValues;
+      case DECIMAL:
+        List<Object> decimalValues = leaf.getLiteralList();
+        decimalValues.replaceAll(value -> BigDecimal.valueOf(((HiveDecimalWritable) value).doubleValue()));
+        return decimalValues;
+      case TIMESTAMP:
+        List<Object> timestampValues = leaf.getLiteralList();
+        timestampValues.replaceAll(value -> ((Timestamp) value).toInstant().getEpochSecond() * MICROS_PER_SECOND +

Review comment:
       same.

##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
##########
@@ -0,0 +1,159 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+
+public class HiveIcebergFilterFactory {
+

Review comment:
       nit: empty line

##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
##########
@@ -0,0 +1,159 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+
+public class HiveIcebergFilterFactory {
+
+  private static final int MICROS_PER_SECOND = 1000000;
+  private static final int NANOSECS_PER_MICROSEC = 1000;
+
+  private HiveIcebergFilterFactory() {}
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(childNodes.get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new IllegalStateException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToLiteral(leaf));
+      case LESS_THAN:
+        return lessThan(column, leafToLiteral(leaf));
+      case LESS_THAN_EQUALS:
+        return lessThanOrEqual(column, leafToLiteral(leaf));
+      case IN:
+        return in(column, leafToLiteralList(leaf));
+      case BETWEEN:
+        List<Object> icebergLiterals = leafToLiteralList(leaf);
+        return and(greaterThanOrEqual(column, icebergLiterals.get(0)),
+                lessThanOrEqual(column, icebergLiterals.get(1)));
+      case IS_NULL:
+        return isNull(column);
+      default:
+        throw new IllegalStateException("Unknown operator: " + leaf.getOperator());
+    }
+  }
+
+  private static Object leafToLiteral(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+      case BOOLEAN:
+      case STRING:
+      case FLOAT:
+        return leaf.getLiteral();
+      case DATE:
+        //Hive converts a Date type to a Timestamp internally when retrieving literal
+        return ((Timestamp) leaf.getLiteral()).toLocalDateTime().toLocalDate().toEpochDay();
+      case DECIMAL:
+        return BigDecimal.valueOf(((HiveDecimalWritable) leaf.getLiteral()).doubleValue());
+      case TIMESTAMP:
+        Timestamp timestamp = (Timestamp) leaf.getLiteral();
+        return timestamp.toInstant().getEpochSecond() * MICROS_PER_SECOND +
+                timestamp.getNanos() / NANOSECS_PER_MICROSEC;
+      default:
+        throw new IllegalStateException("Unknown type: " + leaf.getType());
+    }
+  }
+
+  private static List<Object> leafToLiteralList(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+      case BOOLEAN:
+      case FLOAT:
+      case STRING:
+        return leaf.getLiteralList();
+      case DATE:
+        List<Object> dateValues = leaf.getLiteralList();
+        dateValues.replaceAll(value -> ((Date) value).toLocalDate().toEpochDay());
+        return dateValues;
+      case DECIMAL:
+        List<Object> decimalValues = leaf.getLiteralList();
+        decimalValues.replaceAll(value -> BigDecimal.valueOf(((HiveDecimalWritable) value).doubleValue()));
+        return decimalValues;
+      case TIMESTAMP:
+        List<Object> timestampValues = leaf.getLiteralList();
+        timestampValues.replaceAll(value -> ((Timestamp) value).toInstant().getEpochSecond() * MICROS_PER_SECOND +
+                ((Timestamp) value).getNanos() / NANOSECS_PER_MICROSEC);
+        return timestampValues;
+      default:
+        throw new IllegalStateException("Unknown type: " + leaf.getType());

Review comment:
       nit: better to throw unsupportedexception

##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergInputFormat.java
##########
@@ -51,6 +58,17 @@
 
     forwardConfigSettings(job);

Review comment:
       @cmathiesen the latest `HiveIcebergInputFormat` has changed substantially. Can you please rebase?

##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
##########
@@ -0,0 +1,159 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+
+public class HiveIcebergFilterFactory {
+
+  private static final int MICROS_PER_SECOND = 1000000;
+  private static final int NANOSECS_PER_MICROSEC = 1000;
+
+  private HiveIcebergFilterFactory() {}
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(childNodes.get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new IllegalStateException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToLiteral(leaf));

Review comment:
       seems like leafToLiteral can be called only once and its result used in the switch branches

##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
##########
@@ -0,0 +1,159 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+
+public class HiveIcebergFilterFactory {
+
+  private static final int MICROS_PER_SECOND = 1000000;
+  private static final int NANOSECS_PER_MICROSEC = 1000;

Review comment:
       nit: to b consistent with `MICROS_PER_SECOND` maybe we can rename NANOSECS_PER_MICROSEC to NANOS_PER_MICROSEC

##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
##########
@@ -0,0 +1,159 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+
+public class HiveIcebergFilterFactory {
+
+  private static final int MICROS_PER_SECOND = 1000000;
+  private static final int NANOSECS_PER_MICROSEC = 1000;
+
+  private HiveIcebergFilterFactory() {}
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(childNodes.get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new IllegalStateException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToLiteral(leaf));
+      case LESS_THAN:
+        return lessThan(column, leafToLiteral(leaf));
+      case LESS_THAN_EQUALS:
+        return lessThanOrEqual(column, leafToLiteral(leaf));
+      case IN:
+        return in(column, leafToLiteralList(leaf));
+      case BETWEEN:
+        List<Object> icebergLiterals = leafToLiteralList(leaf);
+        return and(greaterThanOrEqual(column, icebergLiterals.get(0)),
+                lessThanOrEqual(column, icebergLiterals.get(1)));
+      case IS_NULL:
+        return isNull(column);
+      default:
+        throw new IllegalStateException("Unknown operator: " + leaf.getOperator());

Review comment:
       nit: seems like UnuspportedOperationExp is more suitable here

##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
##########
@@ -0,0 +1,159 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+
+public class HiveIcebergFilterFactory {
+
+  private static final int MICROS_PER_SECOND = 1000000;
+  private static final int NANOSECS_PER_MICROSEC = 1000;
+
+  private HiveIcebergFilterFactory() {}
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(childNodes.get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new IllegalStateException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToLiteral(leaf));
+      case LESS_THAN:
+        return lessThan(column, leafToLiteral(leaf));
+      case LESS_THAN_EQUALS:
+        return lessThanOrEqual(column, leafToLiteral(leaf));
+      case IN:
+        return in(column, leafToLiteralList(leaf));
+      case BETWEEN:
+        List<Object> icebergLiterals = leafToLiteralList(leaf);
+        return and(greaterThanOrEqual(column, icebergLiterals.get(0)),
+                lessThanOrEqual(column, icebergLiterals.get(1)));
+      case IS_NULL:
+        return isNull(column);
+      default:
+        throw new IllegalStateException("Unknown operator: " + leaf.getOperator());
+    }
+  }
+
+  private static Object leafToLiteral(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+      case BOOLEAN:
+      case STRING:
+      case FLOAT:
+        return leaf.getLiteral();
+      case DATE:
+        //Hive converts a Date type to a Timestamp internally when retrieving literal
+        return ((Timestamp) leaf.getLiteral()).toLocalDateTime().toLocalDate().toEpochDay();
+      case DECIMAL:
+        return BigDecimal.valueOf(((HiveDecimalWritable) leaf.getLiteral()).doubleValue());

Review comment:
       I'm unsure if this is correct. I think here, the scale of the BigDecimal will always be 0. Irrespective of the underlying data

##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergInputFormat.java
##########
@@ -51,6 +58,17 @@
 
     forwardConfigSettings(job);
 
+    //Convert Hive filter to Iceberg filter
+    String hiveFilter = job.get(TableScanDesc.FILTER_EXPR_CONF_STR);

Review comment:
       I don't think `getSplits` is the right method to set the translated Iceberg expr. I think this should go in `IcebergStorageHandler` . @guilload what do you think?
   
   




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r479703575



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.util.DateTimeUtil;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+
+public class HiveIcebergFilterFactory {
+
+  private HiveIcebergFilterFactory() {
+  }
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(childNodes.get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new UnsupportedOperationException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToLiteral(leaf));
+      case LESS_THAN:
+        return lessThan(column, leafToLiteral(leaf));
+      case LESS_THAN_EQUALS:
+        return lessThanOrEqual(column, leafToLiteral(leaf));
+      case IN:
+        return in(column, leafToLiteralList(leaf));
+      case BETWEEN:
+        List<Object> icebergLiterals = leafToLiteralList(leaf);
+        return and(greaterThanOrEqual(column, icebergLiterals.get(0)),
+                lessThanOrEqual(column, icebergLiterals.get(1)));
+      case IS_NULL:
+        return isNull(column);
+      default:
+        throw new UnsupportedOperationException("Unknown operator: " + leaf.getOperator());
+    }
+  }
+
+  private static Object leafToLiteral(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+      case BOOLEAN:
+      case STRING:
+      case FLOAT:
+        return leaf.getLiteral();
+      case DATE:
+      case TIMESTAMP:
+        // Hive converts a Date type to a Timestamp internally when retrieving literal
+        return timestampToUnixEpoch((Timestamp) leaf.getLiteral());
+      case DECIMAL:
+        return hiveDecimalToBigDecimal((HiveDecimalWritable) leaf.getLiteral());
+
+      default:
+        throw new UnsupportedOperationException("Unknown type: " + leaf.getType());
+    }
+  }
+
+  private static List<Object> leafToLiteralList(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+      case BOOLEAN:
+      case FLOAT:
+      case STRING:
+        return leaf.getLiteralList();
+      case DATE:
+        return leaf.getLiteralList().stream().map(value -> dateToMicros((Date) value))
+                .collect(Collectors.toList());
+      case DECIMAL:
+        return leaf.getLiteralList().stream()
+                .map(value -> hiveDecimalToBigDecimal((HiveDecimalWritable) value))
+                .collect(Collectors.toList());
+      case TIMESTAMP:
+        return leaf.getLiteralList().stream()
+                .map(value -> timestampToUnixEpoch((Timestamp) value))
+                .collect(Collectors.toList());
+      default:
+        throw new UnsupportedOperationException("Unknown type: " + leaf.getType());
+    }
+  }
+
+  private static long dateToMicros(Date date) {

Review comment:
       Dates in Iceberg are the number of days from the unix epoch as an integer, not microseconds. I think this method should be:
   
   ```java
     private static long daysFromDate(Date date) {
       return DateTimeUtil.daysFromDate(date.toLocalDate());
     }
   ```




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdsr commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r476275347



##########
File path: mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergFilterFactory.java
##########
@@ -0,0 +1,243 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentFactory;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.And;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.expressions.Not;
+import org.apache.iceberg.expressions.Or;
+import org.apache.iceberg.expressions.UnboundPredicate;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestHiveIcebergFilterFactory {
+
+  @Test
+  public void testEqualsOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().equals("salary", PredicateLeaf.Type.LONG, 3000L).end().build();
+
+    UnboundPredicate expected = Expressions.equal("salary", 3000L);
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertPredicatesMatch(expected, actual);
+  }
+
+  @Test
+  public void testNotEqualsOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startNot().equals("salary", PredicateLeaf.Type.LONG, 3000L).end().build();
+
+    Not expected = (Not) Expressions.not(Expressions.equal("salary", 3000L));
+    Not actual = (Not) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    UnboundPredicate childExpressionActual = (UnboundPredicate) actual.child();
+    UnboundPredicate childExpressionExpected = Expressions.equal("salary", 3000L);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.child().op(), expected.child().op());
+    assertEquals(childExpressionActual.ref().name(), childExpressionExpected.ref().name());
+    assertEquals(childExpressionActual.literal(), childExpressionExpected.literal());
+  }
+
+  @Test
+  public void testLessThanOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().lessThan("salary", PredicateLeaf.Type.LONG, 3000L).end().build();
+
+    UnboundPredicate expected = Expressions.lessThan("salary", 3000L);
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.literal(), expected.literal());
+    assertEquals(actual.ref().name(), expected.ref().name());
+  }
+
+  @Test
+  public void testLessThanEqualsOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().lessThanEquals("salary", PredicateLeaf.Type.LONG, 3000L).end().build();
+
+    UnboundPredicate expected = Expressions.lessThanOrEqual("salary", 3000L);
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertPredicatesMatch(expected, actual);
+  }
+
+  @Test
+  public void testInOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().in("salary", PredicateLeaf.Type.LONG, 3000L, 4000L).end().build();
+
+    UnboundPredicate expected = Expressions.in("salary", 3000L, 4000L);
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.literals(), expected.literals());
+    assertEquals(actual.ref().name(), expected.ref().name());
+  }
+
+  @Test
+  public void testBetweenOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder
+        .startAnd()
+        .between("salary", PredicateLeaf.Type.LONG, 3000L, 4000L).end().build();
+
+    And expected = (And) Expressions.and(Expressions.greaterThanOrEqual("salary", 3000L),
+        Expressions.lessThanOrEqual("salary", 3000L));
+    And actual = (And) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.left().op(), expected.left().op());
+    assertEquals(actual.right().op(), expected.right().op());
+  }
+
+  @Test
+  public void testIsNullOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().isNull("salary", PredicateLeaf.Type.LONG).end().build();
+
+    UnboundPredicate expected = Expressions.isNull("salary");
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.ref().name(), expected.ref().name());
+  }
+
+  @Test
+  public void testAndOperand() {

Review comment:
       Do we need a test using HiveRunner ? Since Hive stores the table's schema in lowercase I think we might have to support a case insensitive match on the iceberg side.
   cc @pvary, @guilload 




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] guilload commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
guilload commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r473427347



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
##########
@@ -0,0 +1,159 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+
+public class HiveIcebergFilterFactory {
+
+  private static final int MICROS_PER_SECOND = 1000000;
+  private static final int NANOSECS_PER_MICROSEC = 1000;

Review comment:
       What about using `TimeUnit` for the conversion and get rid of those variables altogether?




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] omalley commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
omalley commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r473313825



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergInputFormat.java
##########
@@ -51,6 +58,17 @@
 
     forwardConfigSettings(job);
 
+    //Convert Hive filter to Iceberg filter
+    String hiveFilter = job.get(TableScanDesc.FILTER_EXPR_CONF_STR);

Review comment:
       The most common places to put it are:
   * getSplits - to filter the splits aggressively
   * getRecordReader - to filter the records while reading
   
   The information is being set by HiveInputFormat.pushFilters, but looking through the various paths to get there I think you need to stick to those.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r469395921



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+
+public class HiveIcebergFilterFactory {
+
+  private HiveIcebergFilterFactory() {}
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(tree.getChildren().get(0), leaves));

Review comment:
       Should this be `childNodes.get(0)`?




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r469302984



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+
+public class HiveIcebergFilterFactory {
+
+  private HiveIcebergFilterFactory() {}
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(tree.getChildren().get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        //We are unsure of how the CONSTANT case works, so using the approach of:
+        //https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/
+        // ParquetFilterPredicateConverter.java#L116
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new IllegalStateException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToIcebergType(leaf));
+      case LESS_THAN:
+        return lessThan(column, leafToIcebergType(leaf));
+      case LESS_THAN_EQUALS:
+        return lessThanOrEqual(column, leafToIcebergType(leaf));
+      case IN:
+        return in(column, (List) leafToIcebergType(leaf));
+      case BETWEEN:
+        List<Object> icebergLiterals = leaf.getLiteralList();
+        return and(greaterThanOrEqual(column, icebergLiterals.get(0)),
+                lessThanOrEqual(column, icebergLiterals.get(1)));
+      case IS_NULL:
+        return isNull(column);
+      default:
+        throw new IllegalStateException("Unknown operator: " + leaf.getOperator());
+    }
+  }
+
+  private static Object leafToIcebergType(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+        return leaf.getLiteral() != null ? leaf.getLiteral() : leaf.getLiteralList();
+      case FLOAT:
+        return leaf.getLiteral() != null ? leaf.getLiteral() : leaf.getLiteralList();
+      case STRING:
+        return leaf.getLiteral() != null ? leaf.getLiteral() : leaf.getLiteralList();
+      case DATE:
+        //Hive converts a Date type to a Timestamp internally when retrieving literal
+        if (leaf.getLiteral() != null) {
+          return ((Timestamp) leaf.getLiteral()).toLocalDateTime().toLocalDate().toEpochDay();
+        } else {
+          //But not when retrieving the literalList
+          List<Object> icebergValues = leaf.getLiteralList();
+          icebergValues.replaceAll(value -> ((Date) value).toLocalDate().toEpochDay());
+          return icebergValues;
+        }
+      case DECIMAL:
+        if (leaf.getLiteral() != null) {
+          return BigDecimal.valueOf(((HiveDecimalWritable) leaf.getLiteral()).doubleValue());
+        } else {
+          List<Object> icebergValues = leaf.getLiteralList();
+          icebergValues.replaceAll(value -> BigDecimal.valueOf(((HiveDecimalWritable) value).doubleValue()));
+          return icebergValues;
+        }
+      case TIMESTAMP:
+        if (leaf.getLiteral() != null) {
+          Timestamp timestamp = (Timestamp) leaf.getLiteral();
+          return timestamp.toInstant().getEpochSecond() * 1000000 + timestamp.getNanos() / 1000;

Review comment:
       Not a big deal but I tend to make constants for numbers that can be misread like this, MILLION or NANOS_PER_SECOND




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] cmathiesen commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
cmathiesen commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r474639673



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
##########
@@ -0,0 +1,170 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+
+public class HiveIcebergFilterFactory {
+
+  private HiveIcebergFilterFactory() {}
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(childNodes.get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new UnsupportedOperationException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToLiteral(leaf));
+      case LESS_THAN:
+        return lessThan(column, leafToLiteral(leaf));
+      case LESS_THAN_EQUALS:
+        return lessThanOrEqual(column, leafToLiteral(leaf));
+      case IN:
+        return in(column, leafToLiteralList(leaf));
+      case BETWEEN:
+        List<Object> icebergLiterals = leafToLiteralList(leaf);
+        return and(greaterThanOrEqual(column, icebergLiterals.get(0)),
+                lessThanOrEqual(column, icebergLiterals.get(1)));
+      case IS_NULL:
+        return isNull(column);
+      default:
+        throw new UnsupportedOperationException("Unknown operator: " + leaf.getOperator());
+    }
+  }
+
+  private static Object leafToLiteral(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+      case BOOLEAN:
+      case STRING:
+      case FLOAT:
+        return leaf.getLiteral();
+      case DATE:
+        //Hive converts a Date type to a Timestamp internally when retrieving literal
+        return timestampToDateString((Timestamp) leaf.getLiteral());
+      case DECIMAL:
+        return hiveDecimalToBigDecimal((HiveDecimalWritable) leaf.getLiteral());
+      case TIMESTAMP:
+        return timestampToTimestampString((Timestamp) leaf.getLiteral());
+
+      default:
+        throw new UnsupportedOperationException("Unknown type: " + leaf.getType());
+    }
+  }
+
+  private static List<Object> leafToLiteralList(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+      case BOOLEAN:
+      case FLOAT:
+      case STRING:
+        return leaf.getLiteralList();
+      case DATE:
+        return leaf.getLiteralList().stream().map(value -> dateToString((Date) value))

Review comment:
       I found a small quirk with the Hive Date type where if you call `getLiteral` you get a `Timestamp` back and if you call `getLiteralList` you get `Date` objects, which is why there are 2 separate methods for DATE




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r479703656



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.util.DateTimeUtil;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+
+public class HiveIcebergFilterFactory {
+
+  private HiveIcebergFilterFactory() {
+  }
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(childNodes.get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new UnsupportedOperationException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToLiteral(leaf));
+      case LESS_THAN:
+        return lessThan(column, leafToLiteral(leaf));
+      case LESS_THAN_EQUALS:
+        return lessThanOrEqual(column, leafToLiteral(leaf));
+      case IN:
+        return in(column, leafToLiteralList(leaf));
+      case BETWEEN:
+        List<Object> icebergLiterals = leafToLiteralList(leaf);
+        return and(greaterThanOrEqual(column, icebergLiterals.get(0)),
+                lessThanOrEqual(column, icebergLiterals.get(1)));
+      case IS_NULL:
+        return isNull(column);
+      default:
+        throw new UnsupportedOperationException("Unknown operator: " + leaf.getOperator());
+    }
+  }
+
+  private static Object leafToLiteral(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+      case BOOLEAN:
+      case STRING:
+      case FLOAT:
+        return leaf.getLiteral();
+      case DATE:
+      case TIMESTAMP:
+        // Hive converts a Date type to a Timestamp internally when retrieving literal

Review comment:
       This comment isn't needed.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r469311535



##########
File path: mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergFilterFactory.java
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.
+ */
+

Review comment:
       I think we should probably have tests for all the filter literal types here, It seems like we are only checking Longs?  Especially given the special code around other specific types.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] cmathiesen commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
cmathiesen commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r475508206



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
##########
@@ -140,18 +137,34 @@ private static Object leafToLiteral(PredicateLeaf leaf) {
       case STRING:
         return leaf.getLiteralList();
       case DATE:
-        return leaf.getLiteralList().stream().map(value -> ((Date) value).toLocalDate().toEpochDay())
+        return leaf.getLiteralList().stream().map(value -> dateToString((Date) value))
                 .collect(Collectors.toList());
       case DECIMAL:
         return leaf.getLiteralList().stream()
-                .map(value -> BigDecimal.valueOf(((HiveDecimalWritable) value).doubleValue()))
+                .map(value -> hiveDecimalToBigDecimal((HiveDecimalWritable) value))
                 .collect(Collectors.toList());
       case TIMESTAMP:
         return leaf.getLiteralList().stream()
-                .map(value -> ((Timestamp) value).toInstant().getEpochSecond() * MICROS_PER_SECOND +
-                        ((Timestamp) value).getNanos() / NANOS_PER_MICROSEC).collect(Collectors.toList());
+                .map(value -> timestampToTimestampString((Timestamp) value))

Review comment:
       Ah sure, thank you for explaining that, I think I misunderstood what to do from the last comment - should hopefully be fixed now :)




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] cmathiesen commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
cmathiesen commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r478974677



##########
File path: mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergFilterFactory.java
##########
@@ -0,0 +1,243 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentFactory;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.And;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.expressions.Not;
+import org.apache.iceberg.expressions.Or;
+import org.apache.iceberg.expressions.UnboundPredicate;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestHiveIcebergFilterFactory {
+
+  @Test
+  public void testEqualsOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().equals("salary", PredicateLeaf.Type.LONG, 3000L).end().build();
+
+    UnboundPredicate expected = Expressions.equal("salary", 3000L);
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertPredicatesMatch(expected, actual);
+  }
+
+  @Test
+  public void testNotEqualsOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startNot().equals("salary", PredicateLeaf.Type.LONG, 3000L).end().build();
+
+    Not expected = (Not) Expressions.not(Expressions.equal("salary", 3000L));
+    Not actual = (Not) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    UnboundPredicate childExpressionActual = (UnboundPredicate) actual.child();
+    UnboundPredicate childExpressionExpected = Expressions.equal("salary", 3000L);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.child().op(), expected.child().op());
+    assertEquals(childExpressionActual.ref().name(), childExpressionExpected.ref().name());
+    assertEquals(childExpressionActual.literal(), childExpressionExpected.literal());
+  }
+
+  @Test
+  public void testLessThanOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().lessThan("salary", PredicateLeaf.Type.LONG, 3000L).end().build();
+
+    UnboundPredicate expected = Expressions.lessThan("salary", 3000L);
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.literal(), expected.literal());
+    assertEquals(actual.ref().name(), expected.ref().name());
+  }
+
+  @Test
+  public void testLessThanEqualsOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().lessThanEquals("salary", PredicateLeaf.Type.LONG, 3000L).end().build();
+
+    UnboundPredicate expected = Expressions.lessThanOrEqual("salary", 3000L);
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertPredicatesMatch(expected, actual);
+  }
+
+  @Test
+  public void testInOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().in("salary", PredicateLeaf.Type.LONG, 3000L, 4000L).end().build();
+
+    UnboundPredicate expected = Expressions.in("salary", 3000L, 4000L);
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.literals(), expected.literals());
+    assertEquals(actual.ref().name(), expected.ref().name());
+  }
+
+  @Test
+  public void testBetweenOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder
+        .startAnd()
+        .between("salary", PredicateLeaf.Type.LONG, 3000L, 4000L).end().build();
+
+    And expected = (And) Expressions.and(Expressions.greaterThanOrEqual("salary", 3000L),
+        Expressions.lessThanOrEqual("salary", 3000L));
+    And actual = (And) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.left().op(), expected.left().op());
+    assertEquals(actual.right().op(), expected.right().op());
+  }
+
+  @Test
+  public void testIsNullOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().isNull("salary", PredicateLeaf.Type.LONG).end().build();
+
+    UnboundPredicate expected = Expressions.isNull("salary");
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.ref().name(), expected.ref().name());
+  }
+
+  @Test
+  public void testAndOperand() {

Review comment:
       Sure, sounds good! I think I've addressed all the other comments on this PR so do you have time for another review? @rdblue @rdsr 




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r473216973



##########
File path: mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergFilterFactory.java
##########
@@ -0,0 +1,241 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentFactory;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.And;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.expressions.Not;
+import org.apache.iceberg.expressions.Or;
+import org.apache.iceberg.expressions.UnboundPredicate;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestHiveIcebergFilterFactory {
+
+  @Test
+  public void testEqualsOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().equals("salary", PredicateLeaf.Type.LONG, 3000L).end().build();
+
+    UnboundPredicate expected = Expressions.equal("salary", 3000L);
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertPredicatesMatch(expected, actual);
+  }
+
+  @Test
+  public void testNotEqualsOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startNot().equals("salary", PredicateLeaf.Type.LONG, 3000L).end().build();
+
+    Not expected = (Not) Expressions.not(Expressions.equal("salary", 3000L));
+    Not actual = (Not) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    UnboundPredicate childExpressionActual = (UnboundPredicate) actual.child();
+    UnboundPredicate childExpressionExpected = Expressions.equal("salary", 3000L);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.child().op(), expected.child().op());
+    assertEquals(childExpressionActual.ref().name(), childExpressionExpected.ref().name());
+    assertEquals(childExpressionActual.literal(), childExpressionExpected.literal());
+  }
+
+  @Test
+  public void testLessThanOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().lessThan("salary", PredicateLeaf.Type.LONG, 3000L).end().build();
+
+    UnboundPredicate expected = Expressions.lessThan("salary", 3000L);
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.literal(), expected.literal());
+    assertEquals(actual.ref().name(), expected.ref().name());
+  }
+
+  @Test
+  public void testLessThanEqualsOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().lessThanEquals("salary", PredicateLeaf.Type.LONG, 3000L).end().build();
+
+    UnboundPredicate expected = Expressions.lessThanOrEqual("salary", 3000L);
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertPredicatesMatch(expected, actual);
+  }
+
+  @Test
+  public void testInOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().in("salary", PredicateLeaf.Type.LONG, 3000L, 4000L).end().build();
+
+    UnboundPredicate expected = Expressions.in("salary", 3000L, 4000L);
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.literals(), expected.literals());
+    assertEquals(actual.ref().name(), expected.ref().name());
+  }
+
+  @Test
+  public void testBetweenOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder
+        .startAnd()
+        .between("salary", PredicateLeaf.Type.LONG, 3000L, 4000L).end().build();
+
+    And expected = (And) Expressions.and(Expressions.greaterThanOrEqual("salary", 3000L),
+        Expressions.lessThanOrEqual("salary", 3000L));
+    And actual = (And) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.left().op(), expected.left().op());
+    assertEquals(actual.right().op(), expected.right().op());
+  }
+
+  @Test
+  public void testIsNullOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().isNull("salary", PredicateLeaf.Type.LONG).end().build();
+
+    UnboundPredicate expected = Expressions.isNull("salary");
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.ref().name(), expected.ref().name());
+  }
+
+  @Test
+  public void testAndOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder
+        .startAnd()
+        .equals("salary", PredicateLeaf.Type.LONG, 3000L)
+        .equals("salary", PredicateLeaf.Type.LONG, 4000L)
+        .end().build();
+
+    And expected = (And) Expressions
+        .and(Expressions.equal("salary", 3000L), Expressions.equal("salary", 4000L));
+    And actual = (And) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.left().op(), expected.left().op());
+    assertEquals(actual.right().op(), expected.right().op());
+  }
+
+  @Test
+  public void testOrOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder
+        .startOr()
+        .equals("salary", PredicateLeaf.Type.LONG, 3000L)
+        .equals("salary", PredicateLeaf.Type.LONG, 4000L)
+        .end().build();
+
+    Or expected = (Or) Expressions
+        .or(Expressions.equal("salary", 3000L), Expressions.equal("salary", 4000L));
+    Or actual = (Or) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.left().op(), expected.left().op());
+    assertEquals(actual.right().op(), expected.right().op());
+  }
+
+  @Test
+  public void testStringType() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().equals("string", PredicateLeaf.Type.STRING, "Joe").end().build();
+
+    UnboundPredicate expected = Expressions.equal("string", "Joe");
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertPredicatesMatch(expected, actual);
+  }
+
+  @Test
+  public void testFloatType() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().equals("float", PredicateLeaf.Type.FLOAT, 1200D).end().build();
+
+    UnboundPredicate expected = Expressions.equal("float", 1200D);
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertPredicatesMatch(expected, actual);
+  }
+
+  @Test
+  public void testBooleanType() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().equals("boolean", PredicateLeaf.Type.BOOLEAN, true).end().build();
+
+    UnboundPredicate expected = Expressions.equal("boolean", true);
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertPredicatesMatch(expected, actual);
+  }
+
+  /*@Test
+  public void testDateType() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().equals("date", PredicateLeaf.Type.DATE,
+            Date.valueOf("2015-11-12")).end().build();
+
+    UnboundPredicate expected = Expressions.equal("date", LocalDate.of(2015,11,12));

Review comment:
       Yes, it is a good idea to use a string instead of passing a `LocalDate`. The intent was to avoid tying the API to date/time representations from a specific library.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#issuecomment-683362629


   @cmathiesen, I had a closer look at the date/time and decimal conversion and found that there were a few bugs. I opened https://github.com/ExpediaGroup/iceberg/pull/16 with the fixes for those problems. Could you review that and merge?
   
   One major take-away was that it is not safe to call `Timestamp.toLocalDate` for conversion because that conversion is in local time, not UTC. FYI @massdosage, @rdsr, and @guilload.
   
   This also hits [HIVE-19726](https://issues.apache.org/jira/browse/HIVE-19726), which erases milliseconds. It was fixed in Hive 2.4.0, but I've added a work-around in the PR.


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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r478663135



##########
File path: mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergFilterFactory.java
##########
@@ -0,0 +1,243 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentFactory;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.And;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.expressions.Not;
+import org.apache.iceberg.expressions.Or;
+import org.apache.iceberg.expressions.UnboundPredicate;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestHiveIcebergFilterFactory {
+
+  @Test
+  public void testEqualsOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().equals("salary", PredicateLeaf.Type.LONG, 3000L).end().build();
+
+    UnboundPredicate expected = Expressions.equal("salary", 3000L);
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertPredicatesMatch(expected, actual);
+  }
+
+  @Test
+  public void testNotEqualsOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startNot().equals("salary", PredicateLeaf.Type.LONG, 3000L).end().build();
+
+    Not expected = (Not) Expressions.not(Expressions.equal("salary", 3000L));
+    Not actual = (Not) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    UnboundPredicate childExpressionActual = (UnboundPredicate) actual.child();
+    UnboundPredicate childExpressionExpected = Expressions.equal("salary", 3000L);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.child().op(), expected.child().op());
+    assertEquals(childExpressionActual.ref().name(), childExpressionExpected.ref().name());
+    assertEquals(childExpressionActual.literal(), childExpressionExpected.literal());
+  }
+
+  @Test
+  public void testLessThanOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().lessThan("salary", PredicateLeaf.Type.LONG, 3000L).end().build();
+
+    UnboundPredicate expected = Expressions.lessThan("salary", 3000L);
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.literal(), expected.literal());
+    assertEquals(actual.ref().name(), expected.ref().name());
+  }
+
+  @Test
+  public void testLessThanEqualsOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().lessThanEquals("salary", PredicateLeaf.Type.LONG, 3000L).end().build();
+
+    UnboundPredicate expected = Expressions.lessThanOrEqual("salary", 3000L);
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertPredicatesMatch(expected, actual);
+  }
+
+  @Test
+  public void testInOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().in("salary", PredicateLeaf.Type.LONG, 3000L, 4000L).end().build();
+
+    UnboundPredicate expected = Expressions.in("salary", 3000L, 4000L);
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.literals(), expected.literals());
+    assertEquals(actual.ref().name(), expected.ref().name());
+  }
+
+  @Test
+  public void testBetweenOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder
+        .startAnd()
+        .between("salary", PredicateLeaf.Type.LONG, 3000L, 4000L).end().build();
+
+    And expected = (And) Expressions.and(Expressions.greaterThanOrEqual("salary", 3000L),
+        Expressions.lessThanOrEqual("salary", 3000L));
+    And actual = (And) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.left().op(), expected.left().op());
+    assertEquals(actual.right().op(), expected.right().op());
+  }
+
+  @Test
+  public void testIsNullOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().isNull("salary", PredicateLeaf.Type.LONG).end().build();
+
+    UnboundPredicate expected = Expressions.isNull("salary");
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.ref().name(), expected.ref().name());
+  }
+
+  @Test
+  public void testAndOperand() {

Review comment:
       That doesn't look like a pushdown problem, so I'd open a separate PR to fix it and add the tests.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] cmathiesen commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
cmathiesen commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r472896565



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergInputFormat.java
##########
@@ -51,6 +58,17 @@
 
     forwardConfigSettings(job);
 
+    //Convert Hive filter to Iceberg filter
+    String hiveFilter = job.get(TableScanDesc.FILTER_EXPR_CONF_STR);

Review comment:
       Yeah I think you're right here, and I was initially attempting to do it that way too but I was having some trouble finding the correct method of doing that. I attempted to update the conf in `decomposePredicate` but the way I was doing it caused some bugs where the value I was setting wasn't getting reset between queries if one query had a filter (and decomposePredicate would get called) and the next didn't (so decomposePredicate doesn't get called). And in the other methods like `configureJobConf`, the filter information wasn't available so it couldn't be set. 
   
   I would really appreciate some input on this, I may have missed something with how the StorageHandler works with Hive!




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r473297905



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergInputFormat.java
##########
@@ -51,6 +58,17 @@
 
     forwardConfigSettings(job);

Review comment:
       Agreed. We always merge by squashing the entire PR into a commit, so we do get a linear history in master.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r475828164



##########
File path: mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergFilterFactory.java
##########
@@ -0,0 +1,243 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentFactory;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.And;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.expressions.Not;
+import org.apache.iceberg.expressions.Or;
+import org.apache.iceberg.expressions.UnboundPredicate;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestHiveIcebergFilterFactory {
+
+  @Test
+  public void testEqualsOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().equals("salary", PredicateLeaf.Type.LONG, 3000L).end().build();
+
+    UnboundPredicate expected = Expressions.equal("salary", 3000L);
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertPredicatesMatch(expected, actual);
+  }
+
+  @Test
+  public void testNotEqualsOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startNot().equals("salary", PredicateLeaf.Type.LONG, 3000L).end().build();
+
+    Not expected = (Not) Expressions.not(Expressions.equal("salary", 3000L));
+    Not actual = (Not) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    UnboundPredicate childExpressionActual = (UnboundPredicate) actual.child();
+    UnboundPredicate childExpressionExpected = Expressions.equal("salary", 3000L);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.child().op(), expected.child().op());
+    assertEquals(childExpressionActual.ref().name(), childExpressionExpected.ref().name());
+    assertEquals(childExpressionActual.literal(), childExpressionExpected.literal());
+  }
+
+  @Test
+  public void testLessThanOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().lessThan("salary", PredicateLeaf.Type.LONG, 3000L).end().build();
+
+    UnboundPredicate expected = Expressions.lessThan("salary", 3000L);
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.literal(), expected.literal());
+    assertEquals(actual.ref().name(), expected.ref().name());
+  }
+
+  @Test
+  public void testLessThanEqualsOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().lessThanEquals("salary", PredicateLeaf.Type.LONG, 3000L).end().build();
+
+    UnboundPredicate expected = Expressions.lessThanOrEqual("salary", 3000L);
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertPredicatesMatch(expected, actual);
+  }
+
+  @Test
+  public void testInOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().in("salary", PredicateLeaf.Type.LONG, 3000L, 4000L).end().build();
+
+    UnboundPredicate expected = Expressions.in("salary", 3000L, 4000L);
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.literals(), expected.literals());
+    assertEquals(actual.ref().name(), expected.ref().name());
+  }
+
+  @Test
+  public void testBetweenOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder
+        .startAnd()
+        .between("salary", PredicateLeaf.Type.LONG, 3000L, 4000L).end().build();
+
+    And expected = (And) Expressions.and(Expressions.greaterThanOrEqual("salary", 3000L),
+        Expressions.lessThanOrEqual("salary", 3000L));
+    And actual = (And) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.left().op(), expected.left().op());
+    assertEquals(actual.right().op(), expected.right().op());
+  }
+
+  @Test
+  public void testIsNullOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().isNull("salary", PredicateLeaf.Type.LONG).end().build();
+
+    UnboundPredicate expected = Expressions.isNull("salary");
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.ref().name(), expected.ref().name());
+  }
+
+  @Test
+  public void testAndOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder
+        .startAnd()
+        .equals("salary", PredicateLeaf.Type.LONG, 3000L)
+        .equals("salary", PredicateLeaf.Type.LONG, 4000L)
+        .end().build();
+
+    And expected = (And) Expressions
+        .and(Expressions.equal("salary", 3000L), Expressions.equal("salary", 4000L));
+    And actual = (And) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.left().op(), expected.left().op());
+    assertEquals(actual.right().op(), expected.right().op());
+  }
+
+  @Test
+  public void testOrOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder
+        .startOr()
+        .equals("salary", PredicateLeaf.Type.LONG, 3000L)
+        .equals("salary", PredicateLeaf.Type.LONG, 4000L)
+        .end().build();
+
+    Or expected = (Or) Expressions
+        .or(Expressions.equal("salary", 3000L), Expressions.equal("salary", 4000L));
+    Or actual = (Or) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.left().op(), expected.left().op());
+    assertEquals(actual.right().op(), expected.right().op());
+  }
+
+  @Test
+  public void testStringType() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().equals("string", PredicateLeaf.Type.STRING, "Joe").end().build();
+
+    UnboundPredicate expected = Expressions.equal("string", "Joe");
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertPredicatesMatch(expected, actual);
+  }
+
+  @Test
+  public void testFloatType() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().equals("float", PredicateLeaf.Type.FLOAT, 1200D).end().build();
+
+    UnboundPredicate expected = Expressions.equal("float", 1200D);
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertPredicatesMatch(expected, actual);
+  }
+
+  @Test
+  public void testBooleanType() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().equals("boolean", PredicateLeaf.Type.BOOLEAN, true).end().build();
+
+    UnboundPredicate expected = Expressions.equal("boolean", true);
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertPredicatesMatch(expected, actual);
+  }
+
+  @Test
+  public void testDateType() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().equals("date", PredicateLeaf.Type.DATE,
+            Date.valueOf("2015-11-12")).end().build();
+
+    UnboundPredicate expected = Expressions.equal("date", "2015-11-12");

Review comment:
       I think this expression should use an integer value instead of a String.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] cmathiesen commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
cmathiesen commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r476281343



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+
+public class HiveIcebergFilterFactory {
+
+  private HiveIcebergFilterFactory() {}
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(childNodes.get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new UnsupportedOperationException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToLiteral(leaf));
+      case LESS_THAN:
+        return lessThan(column, leafToLiteral(leaf));
+      case LESS_THAN_EQUALS:
+        return lessThanOrEqual(column, leafToLiteral(leaf));
+      case IN:
+        return in(column, leafToLiteralList(leaf));
+      case BETWEEN:
+        List<Object> icebergLiterals = leafToLiteralList(leaf);
+        return and(greaterThanOrEqual(column, icebergLiterals.get(0)),
+                lessThanOrEqual(column, icebergLiterals.get(1)));
+      case IS_NULL:
+        return isNull(column);
+      default:
+        throw new UnsupportedOperationException("Unknown operator: " + leaf.getOperator());
+    }
+  }
+
+  private static Object leafToLiteral(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+      case BOOLEAN:
+      case STRING:
+      case FLOAT:
+        return leaf.getLiteral();
+      case DATE:
+        //Hive converts a Date type to a Timestamp internally when retrieving literal
+        return timestampToDateString((Timestamp) leaf.getLiteral());
+      case DECIMAL:
+        return hiveDecimalToBigDecimal((HiveDecimalWritable) leaf.getLiteral());
+      case TIMESTAMP:
+        return timestampToUnixEpoch((Timestamp) leaf.getLiteral());
+
+      default:
+        throw new UnsupportedOperationException("Unknown type: " + leaf.getType());
+    }
+  }
+
+  private static List<Object> leafToLiteralList(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+      case BOOLEAN:
+      case FLOAT:
+      case STRING:
+        return leaf.getLiteralList();
+      case DATE:
+        return leaf.getLiteralList().stream().map(value -> dateToString((Date) value))
+                .collect(Collectors.toList());
+      case DECIMAL:
+        return leaf.getLiteralList().stream()
+                .map(value -> hiveDecimalToBigDecimal((HiveDecimalWritable) value))
+                .collect(Collectors.toList());
+      case TIMESTAMP:
+        return leaf.getLiteralList().stream()
+                .map(value -> timestampToUnixEpoch((Timestamp) value))
+                .collect(Collectors.toList());
+      default:
+        throw new UnsupportedOperationException("Unknown type: " + leaf.getType());
+    }
+  }
+
+  private static String timestampToDateString(Timestamp timestamp) {
+    return timestamp.toLocalDateTime().toLocalDate().toString();
+  }
+
+  private static String dateToString(Date date) {
+    return date.toLocalDate().toString();
+  }
+
+  private static BigDecimal hiveDecimalToBigDecimal(HiveDecimalWritable hiveDecimalWritable) {
+    return hiveDecimalWritable.getHiveDecimal().bigDecimalValue().setScale(hiveDecimalWritable.scale());
+  }
+
+  private static long timestampToUnixEpoch(Timestamp timestamp) {
+    return timestamp.toInstant().getEpochSecond() * TimeUnit.SECONDS.toMicros(1) +
+            timestamp.getNanos() / TimeUnit.MICROSECONDS.toNanos(1);

Review comment:
       Ah yep, probably should have spotted that one 😅




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r479703692



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.util.DateTimeUtil;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+
+public class HiveIcebergFilterFactory {
+
+  private HiveIcebergFilterFactory() {
+  }
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(childNodes.get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new UnsupportedOperationException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToLiteral(leaf));
+      case LESS_THAN:
+        return lessThan(column, leafToLiteral(leaf));
+      case LESS_THAN_EQUALS:
+        return lessThanOrEqual(column, leafToLiteral(leaf));
+      case IN:
+        return in(column, leafToLiteralList(leaf));
+      case BETWEEN:
+        List<Object> icebergLiterals = leafToLiteralList(leaf);
+        return and(greaterThanOrEqual(column, icebergLiterals.get(0)),
+                lessThanOrEqual(column, icebergLiterals.get(1)));
+      case IS_NULL:
+        return isNull(column);
+      default:
+        throw new UnsupportedOperationException("Unknown operator: " + leaf.getOperator());
+    }
+  }
+
+  private static Object leafToLiteral(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+      case BOOLEAN:
+      case STRING:
+      case FLOAT:
+        return leaf.getLiteral();
+      case DATE:
+      case TIMESTAMP:
+        // Hive converts a Date type to a Timestamp internally when retrieving literal
+        return timestampToUnixEpoch((Timestamp) leaf.getLiteral());
+      case DECIMAL:
+        return hiveDecimalToBigDecimal((HiveDecimalWritable) leaf.getLiteral());
+
+      default:
+        throw new UnsupportedOperationException("Unknown type: " + leaf.getType());
+    }
+  }
+
+  private static List<Object> leafToLiteralList(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+      case BOOLEAN:
+      case FLOAT:
+      case STRING:
+        return leaf.getLiteralList();
+      case DATE:
+        return leaf.getLiteralList().stream().map(value -> dateToMicros((Date) value))

Review comment:
       Iceberg uses days for a date, not micros, so this should call `daysFromDate`.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdsr commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r472532591



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergInputFormat.java
##########
@@ -51,6 +58,17 @@
 
     forwardConfigSettings(job);
 
+    //Convert Hive filter to Iceberg filter
+    String hiveFilter = job.get(TableScanDesc.FILTER_EXPR_CONF_STR);

Review comment:
       I don't think `getSplits` is the right method to set the translated Iceberg expr in jobconf. I think this should go in `IcebergStorageHandler` . @guilload what do you think?
   
   




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue merged pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
rdblue merged pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326


   


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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#issuecomment-684136693


   Thanks @cmathiesen! Merged. And thanks to all the reviewers!


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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] guilload commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
guilload commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r473435656



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergInputFormat.java
##########
@@ -51,6 +58,17 @@
 
     forwardConfigSettings(job);
 
+    //Convert Hive filter to Iceberg filter
+    String hiveFilter = job.get(TableScanDesc.FILTER_EXPR_CONF_STR);

Review comment:
       I came to the same conclusion when trying to implement projection pushdown in the storage handler.
   
   Unfortunately as @cmathiesen stated, the job config is not yet populated with the projected columns and the filter expression when the storage handler "hooks" such as `configureJobConf` are called. So the right entry points for implementing PPD are `getSplits` and  `getRecordReader`.
   
   ~However, there's another catch. The `JobConf` objects passed in `getSplits` and `getRecordReader` are actually not the same and the filter expression set in `getSplits` (L#53 in `HiveIcebergInputFormat`)  is no longer available when `getRecordReader ` is subsequently called.~
   
   ~Since in the storage handler we don't decompose the filter expression, Hive applies the whole thing anyway and this can't be caught in the test suite but we need to set the filter expression both in `getSplits` and `getRecordReader` to get to a complete PPD implementation.~




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r469301186



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+
+public class HiveIcebergFilterFactory {
+
+  private HiveIcebergFilterFactory() {}
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(tree.getChildren().get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        //We are unsure of how the CONSTANT case works, so using the approach of:
+        //https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/
+        // ParquetFilterPredicateConverter.java#L116
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new IllegalStateException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToIcebergType(leaf));
+      case LESS_THAN:
+        return lessThan(column, leafToIcebergType(leaf));
+      case LESS_THAN_EQUALS:
+        return lessThanOrEqual(column, leafToIcebergType(leaf));
+      case IN:
+        return in(column, (List) leafToIcebergType(leaf));
+      case BETWEEN:
+        List<Object> icebergLiterals = leaf.getLiteralList();
+        return and(greaterThanOrEqual(column, icebergLiterals.get(0)),
+                lessThanOrEqual(column, icebergLiterals.get(1)));
+      case IS_NULL:
+        return isNull(column);
+      default:
+        throw new IllegalStateException("Unknown operator: " + leaf.getOperator());
+    }
+  }
+
+  private static Object leafToIcebergType(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+        return leaf.getLiteral() != null ? leaf.getLiteral() : leaf.getLiteralList();

Review comment:
       I'm new to this code, so I wonder when reading this why we was to get the literal as a list if getLiteral is null? Does having getLiteral() returning null mean that there is a collection type?




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r473222477



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
##########
@@ -0,0 +1,159 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+
+public class HiveIcebergFilterFactory {
+
+  private static final int MICROS_PER_SECOND = 1000000;
+  private static final int NANOSECS_PER_MICROSEC = 1000;
+
+  private HiveIcebergFilterFactory() {}
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(childNodes.get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new IllegalStateException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToLiteral(leaf));
+      case LESS_THAN:
+        return lessThan(column, leafToLiteral(leaf));
+      case LESS_THAN_EQUALS:
+        return lessThanOrEqual(column, leafToLiteral(leaf));
+      case IN:
+        return in(column, leafToLiteralList(leaf));
+      case BETWEEN:
+        List<Object> icebergLiterals = leafToLiteralList(leaf);
+        return and(greaterThanOrEqual(column, icebergLiterals.get(0)),
+                lessThanOrEqual(column, icebergLiterals.get(1)));
+      case IS_NULL:
+        return isNull(column);
+      default:
+        throw new IllegalStateException("Unknown operator: " + leaf.getOperator());
+    }
+  }
+
+  private static Object leafToLiteral(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+      case BOOLEAN:
+      case STRING:
+      case FLOAT:
+        return leaf.getLiteral();
+      case DATE:
+        //Hive converts a Date type to a Timestamp internally when retrieving literal
+        return ((Timestamp) leaf.getLiteral()).toLocalDateTime().toLocalDate().toEpochDay();
+      case DECIMAL:
+        return BigDecimal.valueOf(((HiveDecimalWritable) leaf.getLiteral()).doubleValue());

Review comment:
       Yes, this is not correct because it discards the scale and precision.
   
   This should follow the examples from ORC, which also convert decimals: https://github.com/apache/iceberg/blob/master/data/src/main/java/org/apache/iceberg/data/orc/GenericOrcReaders.java#L163




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] omalley commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
omalley commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r473288060



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergInputFormat.java
##########
@@ -51,6 +58,17 @@
 
     forwardConfigSettings(job);

Review comment:
       That sounds reasonable as long as the PR is rebased to master before it is committed. Having a linear history makes things much much easier to track where a change happened.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r476494526



##########
File path: mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergFilterFactory.java
##########
@@ -0,0 +1,243 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentFactory;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.And;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.expressions.Not;
+import org.apache.iceberg.expressions.Or;
+import org.apache.iceberg.expressions.UnboundPredicate;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestHiveIcebergFilterFactory {
+
+  @Test
+  public void testEqualsOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().equals("salary", PredicateLeaf.Type.LONG, 3000L).end().build();
+
+    UnboundPredicate expected = Expressions.equal("salary", 3000L);
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertPredicatesMatch(expected, actual);
+  }
+
+  @Test
+  public void testNotEqualsOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startNot().equals("salary", PredicateLeaf.Type.LONG, 3000L).end().build();
+
+    Not expected = (Not) Expressions.not(Expressions.equal("salary", 3000L));
+    Not actual = (Not) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    UnboundPredicate childExpressionActual = (UnboundPredicate) actual.child();
+    UnboundPredicate childExpressionExpected = Expressions.equal("salary", 3000L);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.child().op(), expected.child().op());
+    assertEquals(childExpressionActual.ref().name(), childExpressionExpected.ref().name());
+    assertEquals(childExpressionActual.literal(), childExpressionExpected.literal());
+  }
+
+  @Test
+  public void testLessThanOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().lessThan("salary", PredicateLeaf.Type.LONG, 3000L).end().build();
+
+    UnboundPredicate expected = Expressions.lessThan("salary", 3000L);
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.literal(), expected.literal());
+    assertEquals(actual.ref().name(), expected.ref().name());
+  }
+
+  @Test
+  public void testLessThanEqualsOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().lessThanEquals("salary", PredicateLeaf.Type.LONG, 3000L).end().build();
+
+    UnboundPredicate expected = Expressions.lessThanOrEqual("salary", 3000L);
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertPredicatesMatch(expected, actual);
+  }
+
+  @Test
+  public void testInOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().in("salary", PredicateLeaf.Type.LONG, 3000L, 4000L).end().build();
+
+    UnboundPredicate expected = Expressions.in("salary", 3000L, 4000L);
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.literals(), expected.literals());
+    assertEquals(actual.ref().name(), expected.ref().name());
+  }
+
+  @Test
+  public void testBetweenOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder
+        .startAnd()
+        .between("salary", PredicateLeaf.Type.LONG, 3000L, 4000L).end().build();
+
+    And expected = (And) Expressions.and(Expressions.greaterThanOrEqual("salary", 3000L),
+        Expressions.lessThanOrEqual("salary", 3000L));
+    And actual = (And) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.left().op(), expected.left().op());
+    assertEquals(actual.right().op(), expected.right().op());
+  }
+
+  @Test
+  public void testIsNullOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().isNull("salary", PredicateLeaf.Type.LONG).end().build();
+
+    UnboundPredicate expected = Expressions.isNull("salary");
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.ref().name(), expected.ref().name());
+  }
+
+  @Test
+  public void testAndOperand() {

Review comment:
       We definitely need to test for the lowercase column names, since Hive uses that. It might worth to do it for the InputFormat checks as well. On the other hand I am not sure if HiveRunner helps here or not.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r469302984



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+
+public class HiveIcebergFilterFactory {
+
+  private HiveIcebergFilterFactory() {}
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(tree.getChildren().get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        //We are unsure of how the CONSTANT case works, so using the approach of:
+        //https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/
+        // ParquetFilterPredicateConverter.java#L116
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new IllegalStateException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToIcebergType(leaf));
+      case LESS_THAN:
+        return lessThan(column, leafToIcebergType(leaf));
+      case LESS_THAN_EQUALS:
+        return lessThanOrEqual(column, leafToIcebergType(leaf));
+      case IN:
+        return in(column, (List) leafToIcebergType(leaf));
+      case BETWEEN:
+        List<Object> icebergLiterals = leaf.getLiteralList();
+        return and(greaterThanOrEqual(column, icebergLiterals.get(0)),
+                lessThanOrEqual(column, icebergLiterals.get(1)));
+      case IS_NULL:
+        return isNull(column);
+      default:
+        throw new IllegalStateException("Unknown operator: " + leaf.getOperator());
+    }
+  }
+
+  private static Object leafToIcebergType(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+        return leaf.getLiteral() != null ? leaf.getLiteral() : leaf.getLiteralList();
+      case FLOAT:
+        return leaf.getLiteral() != null ? leaf.getLiteral() : leaf.getLiteralList();
+      case STRING:
+        return leaf.getLiteral() != null ? leaf.getLiteral() : leaf.getLiteralList();
+      case DATE:
+        //Hive converts a Date type to a Timestamp internally when retrieving literal
+        if (leaf.getLiteral() != null) {
+          return ((Timestamp) leaf.getLiteral()).toLocalDateTime().toLocalDate().toEpochDay();
+        } else {
+          //But not when retrieving the literalList
+          List<Object> icebergValues = leaf.getLiteralList();
+          icebergValues.replaceAll(value -> ((Date) value).toLocalDate().toEpochDay());
+          return icebergValues;
+        }
+      case DECIMAL:
+        if (leaf.getLiteral() != null) {
+          return BigDecimal.valueOf(((HiveDecimalWritable) leaf.getLiteral()).doubleValue());
+        } else {
+          List<Object> icebergValues = leaf.getLiteralList();
+          icebergValues.replaceAll(value -> BigDecimal.valueOf(((HiveDecimalWritable) value).doubleValue()));
+          return icebergValues;
+        }
+      case TIMESTAMP:
+        if (leaf.getLiteral() != null) {
+          Timestamp timestamp = (Timestamp) leaf.getLiteral();
+          return timestamp.toInstant().getEpochSecond() * 1000000 + timestamp.getNanos() / 1000;

Review comment:
       Not a big deal but I tend to make constants for numbers that can be misread like this, MILLION or MICROS_PER_SECOND




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] cmathiesen commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
cmathiesen commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r478433463



##########
File path: mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergFilterFactory.java
##########
@@ -0,0 +1,243 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentFactory;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.And;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.expressions.Not;
+import org.apache.iceberg.expressions.Or;
+import org.apache.iceberg.expressions.UnboundPredicate;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestHiveIcebergFilterFactory {
+
+  @Test
+  public void testEqualsOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().equals("salary", PredicateLeaf.Type.LONG, 3000L).end().build();
+
+    UnboundPredicate expected = Expressions.equal("salary", 3000L);
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertPredicatesMatch(expected, actual);
+  }
+
+  @Test
+  public void testNotEqualsOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startNot().equals("salary", PredicateLeaf.Type.LONG, 3000L).end().build();
+
+    Not expected = (Not) Expressions.not(Expressions.equal("salary", 3000L));
+    Not actual = (Not) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    UnboundPredicate childExpressionActual = (UnboundPredicate) actual.child();
+    UnboundPredicate childExpressionExpected = Expressions.equal("salary", 3000L);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.child().op(), expected.child().op());
+    assertEquals(childExpressionActual.ref().name(), childExpressionExpected.ref().name());
+    assertEquals(childExpressionActual.literal(), childExpressionExpected.literal());
+  }
+
+  @Test
+  public void testLessThanOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().lessThan("salary", PredicateLeaf.Type.LONG, 3000L).end().build();
+
+    UnboundPredicate expected = Expressions.lessThan("salary", 3000L);
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.literal(), expected.literal());
+    assertEquals(actual.ref().name(), expected.ref().name());
+  }
+
+  @Test
+  public void testLessThanEqualsOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().lessThanEquals("salary", PredicateLeaf.Type.LONG, 3000L).end().build();
+
+    UnboundPredicate expected = Expressions.lessThanOrEqual("salary", 3000L);
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertPredicatesMatch(expected, actual);
+  }
+
+  @Test
+  public void testInOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().in("salary", PredicateLeaf.Type.LONG, 3000L, 4000L).end().build();
+
+    UnboundPredicate expected = Expressions.in("salary", 3000L, 4000L);
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.literals(), expected.literals());
+    assertEquals(actual.ref().name(), expected.ref().name());
+  }
+
+  @Test
+  public void testBetweenOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder
+        .startAnd()
+        .between("salary", PredicateLeaf.Type.LONG, 3000L, 4000L).end().build();
+
+    And expected = (And) Expressions.and(Expressions.greaterThanOrEqual("salary", 3000L),
+        Expressions.lessThanOrEqual("salary", 3000L));
+    And actual = (And) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.left().op(), expected.left().op());
+    assertEquals(actual.right().op(), expected.right().op());
+  }
+
+  @Test
+  public void testIsNullOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().isNull("salary", PredicateLeaf.Type.LONG).end().build();
+
+    UnboundPredicate expected = Expressions.isNull("salary");
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.ref().name(), expected.ref().name());
+  }
+
+  @Test
+  public void testAndOperand() {

Review comment:
       I've been working on a HiveRunner test to see what happens in this case: 
   
   I've got an Iceberg table with a schema like: 
   ```
   private static final Schema STOCK_LIST_SCHEMA = new Schema(
             required(1, "ITEM_ID", Types.LongType.get()),
             required(2, "ITEM_COUNT", Types.LongType.get())
     );
   ```
   
   If I run a regular query either like `SELECT ITEM_ID from default.stock_table` or `SELECT item_id from default.stock_table` then this error occurs:
   
   ```
   Caused by: java.lang.RuntimeException: cannot find field item_id from [org.apache.iceberg.mr.hive.serde.objectinspector.IcebergRecordObjectInspector$IcebergRecordStructField@c0fc462a, org.apache.iceberg.mr.hive.serde.objectinspector.IcebergRecordObjectInspector$IcebergRecordStructField@275a564e]
   at org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.getStandardStructFieldRef(ObjectInspectorUtils.java:523)
   at org.apache.iceberg.mr.hive.serde.objectinspector.IcebergRecordObjectInspector.getStructFieldRef(IcebergRecordObjectInspector.java:68)
   at org.apache.hadoop.hive.ql.exec.ExprNodeColumnEvaluator.initialize(ExprNodeColumnEvaluator.java:56)
   at org.apache.hadoop.hive.ql.exec.Operator.initEvaluators(Operator.java:1033)
   at org.apache.hadoop.hive.ql.exec.Operator.initEvaluatorsAndReturnStruct(Operator.java:1059)
   at org.apache.hadoop.hive.ql.exec.SelectOperator.initializeOp(SelectOperator.java:75)
   at org.apache.hadoop.hive.ql.exec.Operator.initialize(Operator.java:366)
   at org.apache.hadoop.hive.ql.exec.Operator.initialize(Operator.java:556)
   at org.apache.hadoop.hive.ql.exec.Operator.initializeChildren(Operator.java:508)
   at org.apache.hadoop.hive.ql.exec.Operator.initialize(Operator.java:376)
   at org.apache.hadoop.hive.ql.exec.FetchTask.initialize(FetchTask.java:88)
   ... 29 more
   ```
   which looks like the case sensitivity issues @rdsr mentioned.
   
   I haven't pushed this test yet but I can do so if others want to reproduce the issue (I've just added a test to `HiveIcebergStorageHandlerBaseTest`).
   
   Where would be the best place to put in a fix for this? This also doesn't rely on predicate pushdown so it could be done in another PR if needed




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] cmathiesen commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
cmathiesen commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r473004549



##########
File path: mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergFilterFactory.java
##########
@@ -0,0 +1,241 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentFactory;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.And;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.expressions.Not;
+import org.apache.iceberg.expressions.Or;
+import org.apache.iceberg.expressions.UnboundPredicate;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestHiveIcebergFilterFactory {
+
+  @Test
+  public void testEqualsOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().equals("salary", PredicateLeaf.Type.LONG, 3000L).end().build();
+
+    UnboundPredicate expected = Expressions.equal("salary", 3000L);
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertPredicatesMatch(expected, actual);
+  }
+
+  @Test
+  public void testNotEqualsOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startNot().equals("salary", PredicateLeaf.Type.LONG, 3000L).end().build();
+
+    Not expected = (Not) Expressions.not(Expressions.equal("salary", 3000L));
+    Not actual = (Not) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    UnboundPredicate childExpressionActual = (UnboundPredicate) actual.child();
+    UnboundPredicate childExpressionExpected = Expressions.equal("salary", 3000L);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.child().op(), expected.child().op());
+    assertEquals(childExpressionActual.ref().name(), childExpressionExpected.ref().name());
+    assertEquals(childExpressionActual.literal(), childExpressionExpected.literal());
+  }
+
+  @Test
+  public void testLessThanOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().lessThan("salary", PredicateLeaf.Type.LONG, 3000L).end().build();
+
+    UnboundPredicate expected = Expressions.lessThan("salary", 3000L);
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.literal(), expected.literal());
+    assertEquals(actual.ref().name(), expected.ref().name());
+  }
+
+  @Test
+  public void testLessThanEqualsOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().lessThanEquals("salary", PredicateLeaf.Type.LONG, 3000L).end().build();
+
+    UnboundPredicate expected = Expressions.lessThanOrEqual("salary", 3000L);
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertPredicatesMatch(expected, actual);
+  }
+
+  @Test
+  public void testInOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().in("salary", PredicateLeaf.Type.LONG, 3000L, 4000L).end().build();
+
+    UnboundPredicate expected = Expressions.in("salary", 3000L, 4000L);
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.literals(), expected.literals());
+    assertEquals(actual.ref().name(), expected.ref().name());
+  }
+
+  @Test
+  public void testBetweenOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder
+        .startAnd()
+        .between("salary", PredicateLeaf.Type.LONG, 3000L, 4000L).end().build();
+
+    And expected = (And) Expressions.and(Expressions.greaterThanOrEqual("salary", 3000L),
+        Expressions.lessThanOrEqual("salary", 3000L));
+    And actual = (And) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.left().op(), expected.left().op());
+    assertEquals(actual.right().op(), expected.right().op());
+  }
+
+  @Test
+  public void testIsNullOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().isNull("salary", PredicateLeaf.Type.LONG).end().build();
+
+    UnboundPredicate expected = Expressions.isNull("salary");
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.ref().name(), expected.ref().name());
+  }
+
+  @Test
+  public void testAndOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder
+        .startAnd()
+        .equals("salary", PredicateLeaf.Type.LONG, 3000L)
+        .equals("salary", PredicateLeaf.Type.LONG, 4000L)
+        .end().build();
+
+    And expected = (And) Expressions
+        .and(Expressions.equal("salary", 3000L), Expressions.equal("salary", 4000L));
+    And actual = (And) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.left().op(), expected.left().op());
+    assertEquals(actual.right().op(), expected.right().op());
+  }
+
+  @Test
+  public void testOrOperand() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder
+        .startOr()
+        .equals("salary", PredicateLeaf.Type.LONG, 3000L)
+        .equals("salary", PredicateLeaf.Type.LONG, 4000L)
+        .end().build();
+
+    Or expected = (Or) Expressions
+        .or(Expressions.equal("salary", 3000L), Expressions.equal("salary", 4000L));
+    Or actual = (Or) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertEquals(actual.op(), expected.op());
+    assertEquals(actual.left().op(), expected.left().op());
+    assertEquals(actual.right().op(), expected.right().op());
+  }
+
+  @Test
+  public void testStringType() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().equals("string", PredicateLeaf.Type.STRING, "Joe").end().build();
+
+    UnboundPredicate expected = Expressions.equal("string", "Joe");
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertPredicatesMatch(expected, actual);
+  }
+
+  @Test
+  public void testFloatType() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().equals("float", PredicateLeaf.Type.FLOAT, 1200D).end().build();
+
+    UnboundPredicate expected = Expressions.equal("float", 1200D);
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertPredicatesMatch(expected, actual);
+  }
+
+  @Test
+  public void testBooleanType() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().equals("boolean", PredicateLeaf.Type.BOOLEAN, true).end().build();
+
+    UnboundPredicate expected = Expressions.equal("boolean", true);
+    UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg);
+
+    assertPredicatesMatch(expected, actual);
+  }
+
+  /*@Test
+  public void testDateType() {
+    SearchArgument.Builder builder = SearchArgumentFactory.newBuilder();
+    SearchArgument arg = builder.startAnd().equals("date", PredicateLeaf.Type.DATE,
+            Date.valueOf("2015-11-12")).end().build();
+
+    UnboundPredicate expected = Expressions.equal("date", LocalDate.of(2015,11,12));

Review comment:
       @rdblue @rdsr I added tests for the `Date` and `Timestamp` types but when these are run I get errors like: 
   
   ```
   java.lang.IllegalArgumentException: Cannot create expression literal from java.time.LocalDate: 2015-11-12
   	at org.apache.iceberg.expressions.Literals.from(Literals.java:83)
   	at org.apache.iceberg.expressions.UnboundPredicate.<init>(UnboundPredicate.java:39)
   	at org.apache.iceberg.expressions.Expressions.equal(Expressions.java:159)
           at org.apache.iceberg.mr.hive.TestHiveIcebergFilterFactory.testDateType(TestHiveIcebergFilterFactory.java:211)
   ```
   
   I noticed [here](https://github.com/apache/iceberg/blob/c20927801a369104e5ea510470e1cf7c8e28b808/api/src/test/java/org/apache/iceberg/expressions/TestLiteralSerialization.java#L38) in another test that Date's etc.  are actually passed as Strings - is that the correct option to be using in this 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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] cmathiesen commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
cmathiesen commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r474639018



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+
+public class HiveIcebergFilterFactory {
+  private static final int MICROS_PER_SECOND = 1000000;
+  private static final int NANOS_PER_MICROSEC = 1000;
+
+  private HiveIcebergFilterFactory() {}
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(childNodes.get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new UnsupportedOperationException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToLiteral(leaf));
+      case LESS_THAN:
+        return lessThan(column, leafToLiteral(leaf));
+      case LESS_THAN_EQUALS:
+        return lessThanOrEqual(column, leafToLiteral(leaf));
+      case IN:
+        return in(column, leafToLiteralList(leaf));
+      case BETWEEN:
+        List<Object> icebergLiterals = leafToLiteralList(leaf);

Review comment:
       I believe that we can expect there to be 2, as we're using the BETWEEN operator and Hive wouldn't accept more than 2 arguments




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r469311535



##########
File path: mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergFilterFactory.java
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.
+ */
+

Review comment:
       I think we should probably have tests for all the filter types here, It seems like we are only checking Longs?  Especially given the special code around other specific types.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r475826966



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+
+public class HiveIcebergFilterFactory {
+
+  private HiveIcebergFilterFactory() {}
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(childNodes.get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new UnsupportedOperationException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToLiteral(leaf));
+      case LESS_THAN:
+        return lessThan(column, leafToLiteral(leaf));
+      case LESS_THAN_EQUALS:
+        return lessThanOrEqual(column, leafToLiteral(leaf));
+      case IN:
+        return in(column, leafToLiteralList(leaf));
+      case BETWEEN:
+        List<Object> icebergLiterals = leafToLiteralList(leaf);
+        return and(greaterThanOrEqual(column, icebergLiterals.get(0)),
+                lessThanOrEqual(column, icebergLiterals.get(1)));
+      case IS_NULL:
+        return isNull(column);
+      default:
+        throw new UnsupportedOperationException("Unknown operator: " + leaf.getOperator());
+    }
+  }
+
+  private static Object leafToLiteral(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+      case BOOLEAN:
+      case STRING:
+      case FLOAT:
+        return leaf.getLiteral();
+      case DATE:
+        //Hive converts a Date type to a Timestamp internally when retrieving literal
+        return timestampToDateString((Timestamp) leaf.getLiteral());
+      case DECIMAL:
+        return hiveDecimalToBigDecimal((HiveDecimalWritable) leaf.getLiteral());
+      case TIMESTAMP:
+        return timestampToUnixEpoch((Timestamp) leaf.getLiteral());
+
+      default:
+        throw new UnsupportedOperationException("Unknown type: " + leaf.getType());
+    }
+  }
+
+  private static List<Object> leafToLiteralList(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+      case BOOLEAN:
+      case FLOAT:
+      case STRING:
+        return leaf.getLiteralList();
+      case DATE:
+        return leaf.getLiteralList().stream().map(value -> dateToString((Date) value))
+                .collect(Collectors.toList());
+      case DECIMAL:
+        return leaf.getLiteralList().stream()
+                .map(value -> hiveDecimalToBigDecimal((HiveDecimalWritable) value))
+                .collect(Collectors.toList());
+      case TIMESTAMP:
+        return leaf.getLiteralList().stream()
+                .map(value -> timestampToUnixEpoch((Timestamp) value))
+                .collect(Collectors.toList());
+      default:
+        throw new UnsupportedOperationException("Unknown type: " + leaf.getType());
+    }
+  }
+
+  private static String timestampToDateString(Timestamp timestamp) {
+    return timestamp.toLocalDateTime().toLocalDate().toString();
+  }
+
+  private static String dateToString(Date date) {
+    return date.toLocalDate().toString();
+  }
+
+  private static BigDecimal hiveDecimalToBigDecimal(HiveDecimalWritable hiveDecimalWritable) {
+    return hiveDecimalWritable.getHiveDecimal().bigDecimalValue().setScale(hiveDecimalWritable.scale());
+  }
+
+  private static long timestampToUnixEpoch(Timestamp timestamp) {
+    return timestamp.toInstant().getEpochSecond() * TimeUnit.SECONDS.toMicros(1) +
+            timestamp.getNanos() / TimeUnit.MICROSECONDS.toNanos(1);

Review comment:
       This seems odd to me. Why not call `TimeUnit.SECONDS.toMicros(timestamp.toInstant().getEpochSecond())`? Using the `toMicros` function to get the conversion factor, but not actually using it for conversion is strange.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r473217813



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+
+public class HiveIcebergFilterFactory {
+  private static final int MICROS_PER_SECOND = 1000000;
+  private static final int NANOS_PER_MICROSEC = 1000;
+
+  private HiveIcebergFilterFactory() {}
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(childNodes.get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new UnsupportedOperationException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToLiteral(leaf));
+      case LESS_THAN:
+        return lessThan(column, leafToLiteral(leaf));
+      case LESS_THAN_EQUALS:
+        return lessThanOrEqual(column, leafToLiteral(leaf));
+      case IN:
+        return in(column, leafToLiteralList(leaf));
+      case BETWEEN:
+        List<Object> icebergLiterals = leafToLiteralList(leaf);

Review comment:
       Do we need to validate that there are only two literals here, or is this reliable?




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r475196226



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergInputFormat.java
##########
@@ -36,6 +43,16 @@
 
   @Override
   public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
+    //Convert Hive filter to Iceberg filter
+    String hiveFilter = job.get(TableScanDesc.FILTER_EXPR_CONF_STR);
+    if (hiveFilter != null) {
+      ExprNodeGenericFuncDesc exprNodeDesc = SerializationUtilities
+              .deserializeObject(hiveFilter, ExprNodeGenericFuncDesc.class);
+      SearchArgument sarg = ConvertAstToSearchArg.create(job, exprNodeDesc);
+      Expression filter = HiveIcebergFilterFactory.generateFilterExpression(sarg);

Review comment:
       HiveIcebergFilterFactory.generateFilterExpression might throw UnsupportedOperationException.
   Maybe it would be good to catch the exception and continue without filters in case if there is an error.
   Hive runs the filters later anyway, so it will not cause issue.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] guilload commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
guilload commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r473486618



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergInputFormat.java
##########
@@ -51,6 +58,17 @@
 
     forwardConfigSettings(job);
 
+    //Convert Hive filter to Iceberg filter
+    String hiveFilter = job.get(TableScanDesc.FILTER_EXPR_CONF_STR);

Review comment:
       and looking at the code, the underlying MR input format already does the right thing and passes the residual filter from the task down to the reader. Great! My comment is only relevant for projection pushdown then. My bad.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r473215871



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergInputFormat.java
##########
@@ -51,6 +58,17 @@
 
     forwardConfigSettings(job);

Review comment:
       I think that either rebasing or merging master into a PR is okay.
   
   As a reviewer, I don't really consider PR branches to be public because github handles force-pushes well. If I have a PR checked out, I also don't mind resetting to the PR's current state because I like keeping history clean.
   
   That said, if you're sharing a PR branch between people that can be disruptive, so I think it is up to the author and collaborators whether to merge or to rebase to stay up to date with master.
   
   Is that reasonable?




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] massdosage commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
massdosage commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r472991930



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergInputFormat.java
##########
@@ -51,6 +58,17 @@
 
     forwardConfigSettings(job);

Review comment:
       > rebase
   
   Shouldn't we be following the [Golden Rule of Rebasing](https://www.atlassian.com/git/tutorials/merging-vs-rebasing#the-golden-rule-of-rebasing) and *not* do this on public branches? It has the potential to cause all kinds of inconsistencies on other people's checkouts. Surely we should be doing merge? It all gets squash merged at the end so having a pristine history isn't worth the downsides of inconsistencies IHMO.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] cmathiesen commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
cmathiesen commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r469377160



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+
+public class HiveIcebergFilterFactory {
+
+  private HiveIcebergFilterFactory() {}
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(tree.getChildren().get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        //We are unsure of how the CONSTANT case works, so using the approach of:
+        //https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/
+        // ParquetFilterPredicateConverter.java#L116
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new IllegalStateException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToIcebergType(leaf));
+      case LESS_THAN:
+        return lessThan(column, leafToIcebergType(leaf));
+      case LESS_THAN_EQUALS:
+        return lessThanOrEqual(column, leafToIcebergType(leaf));
+      case IN:
+        return in(column, (List) leafToIcebergType(leaf));
+      case BETWEEN:
+        List<Object> icebergLiterals = leaf.getLiteralList();
+        return and(greaterThanOrEqual(column, icebergLiterals.get(0)),
+                lessThanOrEqual(column, icebergLiterals.get(1)));
+      case IS_NULL:
+        return isNull(column);
+      default:
+        throw new IllegalStateException("Unknown operator: " + leaf.getOperator());
+    }
+  }
+
+  private static Object leafToIcebergType(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+        return leaf.getLiteral() != null ? leaf.getLiteral() : leaf.getLiteralList();

Review comment:
       Yeah, that's sort of whats going on. [It looks like](https://github.com/apache/hive/blob/master/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/PredicateLeaf.java#L96) you'd only be using `getLiteralList` if the operator for the leaf was `IN` or `BETWEEN`, and then `getLiteral` for all other operator types. It would either be one or the other, so it seemed easiest to check for a null rather than calling `getOperator` and having cases to switch through all the different operators, if that makes sense




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r473443420



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergInputFormat.java
##########
@@ -51,6 +58,17 @@
 
     forwardConfigSettings(job);
 
+    //Convert Hive filter to Iceberg filter
+    String hiveFilter = job.get(TableScanDesc.FILTER_EXPR_CONF_STR);

Review comment:
       Can't the record reader use the residuals that are present in each task produced by `getSplits`? I don't think we need to get the filter twice as long as `getSplits` is using it and creating tasks with residuals.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r473224375



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergInputFormat.java
##########
@@ -51,6 +58,17 @@
 
     forwardConfigSettings(job);
 
+    //Convert Hive filter to Iceberg filter
+    String hiveFilter = job.get(TableScanDesc.FILTER_EXPR_CONF_STR);

Review comment:
       @omalley, any input on where the filter pushdown should go?




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r478278194



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.util.DateTimeUtil;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+
+public class HiveIcebergFilterFactory {
+
+  private HiveIcebergFilterFactory() {
+  }
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());

Review comment:
       Maybe logging would be nice here minimally on DEBUG level, but maybe on INFO level, like:
   ```
   LOG.info("Translated sarg=[{}] to expression=[{}]", sarg, expression);
   ```
   Not sure about the toString implementations, but the general idea would be to see what went in and what came out.
   Also we can add this later, just noting here so we do not forget :D




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] guilload commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
guilload commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r473470990



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergInputFormat.java
##########
@@ -51,6 +58,17 @@
 
     forwardConfigSettings(job);
 
+    //Convert Hive filter to Iceberg filter
+    String hiveFilter = job.get(TableScanDesc.FILTER_EXPR_CONF_STR);

Review comment:
       Oh very good point. Tasks do carry the residual filter.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r469298303



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+
+public class HiveIcebergFilterFactory {
+
+  private HiveIcebergFilterFactory() {}
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(tree.getChildren().get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        //We are unsure of how the CONSTANT case works, so using the approach of:
+        //https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/
+        // ParquetFilterPredicateConverter.java#L116

Review comment:
       This comment seems to be at odds with the exception being thrown? It looks like in the hive code it just does nothing? Maybe I'm reading it wrong.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] massdosage commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
massdosage commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r472922327



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
##########
@@ -0,0 +1,159 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+
+public class HiveIcebergFilterFactory {
+
+  private static final int MICROS_PER_SECOND = 1000000;
+  private static final int NANOSECS_PER_MICROSEC = 1000;
+
+  private HiveIcebergFilterFactory() {}
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(childNodes.get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new IllegalStateException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToLiteral(leaf));

Review comment:
       > seems like leafToLiteral can be called only once and its result used in the switch branches
   
   But `leafToLiteral` isn't used in all the switch branches (e.g. `IN` and `BETWEEN` use `leafToLiteralList`, `IS_NULL` doesn't use it at all) so we'd be calling it for nothing in those cases. I think it's fine as is?




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] guilload commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
guilload commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r473435656



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergInputFormat.java
##########
@@ -51,6 +58,17 @@
 
     forwardConfigSettings(job);
 
+    //Convert Hive filter to Iceberg filter
+    String hiveFilter = job.get(TableScanDesc.FILTER_EXPR_CONF_STR);

Review comment:
       I came to the same conclusion when trying to implement projection pushdown in the storage handler.
   
   Unfortunately as @cmathiesen stated, the job config is not yet populated with the projected columns and the filter expression when the storage handler "hooks" such as `configureJobConf` are called. So the right entry points for implementing PPD are `getSplits` and  `getRecordReader`.
   
   However, there's another catch. The `JobConf` objects passed in `getSplits` and `getRecordReader` are actually not the same and the filter expression set in `getSplits` (L#53 in `HiveIcebergInputFormat`)  is no longer available when `getRecordReader ` is subsequently called.
   
   Since in the storage handler we don't decompose the filter expression, Hive applies the whole thing anyway and this can't be caught in the test suite but we need to set the filter expression both in `getSplits` and `getRecordReader` to get to a complete PPD implementation.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r478662375



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.util.DateTimeUtil;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+
+public class HiveIcebergFilterFactory {
+
+  private HiveIcebergFilterFactory() {
+  }
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());

Review comment:
       Pushed filters are logged in the scan, so the translated expression is already logged. I assume that Hive also logs the filters that it is pushing, so I don't think this is necessary.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r473223699



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+
+public class HiveIcebergFilterFactory {
+  private static final int MICROS_PER_SECOND = 1000000;
+  private static final int NANOS_PER_MICROSEC = 1000;
+
+  private HiveIcebergFilterFactory() {}
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(childNodes.get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new UnsupportedOperationException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToLiteral(leaf));
+      case LESS_THAN:
+        return lessThan(column, leafToLiteral(leaf));
+      case LESS_THAN_EQUALS:
+        return lessThanOrEqual(column, leafToLiteral(leaf));
+      case IN:
+        return in(column, leafToLiteralList(leaf));
+      case BETWEEN:
+        List<Object> icebergLiterals = leafToLiteralList(leaf);
+        return and(greaterThanOrEqual(column, icebergLiterals.get(0)),
+                lessThanOrEqual(column, icebergLiterals.get(1)));
+      case IS_NULL:
+        return isNull(column);
+      default:
+        throw new UnsupportedOperationException("Unknown operator: " + leaf.getOperator());
+    }
+  }
+
+  private static Object leafToLiteral(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+      case BOOLEAN:
+      case STRING:
+      case FLOAT:
+        return leaf.getLiteral();
+      case DATE:
+        //Hive converts a Date type to a Timestamp internally when retrieving literal
+        return ((Timestamp) leaf.getLiteral()).toLocalDateTime().toLocalDate().toEpochDay();
+      case DECIMAL:
+        return BigDecimal.valueOf(((HiveDecimalWritable) leaf.getLiteral()).doubleValue());
+      case TIMESTAMP:
+        Timestamp timestamp = (Timestamp) leaf.getLiteral();
+        return timestamp.toInstant().getEpochSecond() * MICROS_PER_SECOND +
+                timestamp.getNanos() / NANOS_PER_MICROSEC;
+      default:
+        throw new IllegalStateException("Unknown type: " + leaf.getType());
+    }
+  }
+
+  private static List<Object> leafToLiteralList(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+      case BOOLEAN:
+      case FLOAT:
+      case STRING:
+        return leaf.getLiteralList();
+      case DATE:
+        return leaf.getLiteralList().stream().map(value -> ((Date) value).toLocalDate().toEpochDay())
+                .collect(Collectors.toList());
+      case DECIMAL:
+        return leaf.getLiteralList().stream()
+                .map(value -> BigDecimal.valueOf(((HiveDecimalWritable) value).doubleValue()))
+                .collect(Collectors.toList());

Review comment:
       Could these cases share the conversion logic between `leafToLiteral` and `leafToLiteralList`? That way if we need to fix something it is always applied to both cases.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] massdosage commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
massdosage commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r472922327



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
##########
@@ -0,0 +1,159 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+
+public class HiveIcebergFilterFactory {
+
+  private static final int MICROS_PER_SECOND = 1000000;
+  private static final int NANOSECS_PER_MICROSEC = 1000;
+
+  private HiveIcebergFilterFactory() {}
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(childNodes.get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new IllegalStateException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToLiteral(leaf));

Review comment:
       > seems like leafToLiteral can be called only once and its result used in the switch branches
   
   But `leafToLiteral` isn't used in all the switch branches (e.g. `IN` and `BETWEEN` use uses `leafToLiteralList`, `IS_NULL` doesn't use it at all) so we'd be calling it for nothing in those cases. I think it's fine as is?




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r469402468



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+
+public class HiveIcebergFilterFactory {
+
+  private HiveIcebergFilterFactory() {}
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(tree.getChildren().get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        //We are unsure of how the CONSTANT case works, so using the approach of:
+        //https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/
+        // ParquetFilterPredicateConverter.java#L116
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new IllegalStateException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToIcebergType(leaf));
+      case LESS_THAN:
+        return lessThan(column, leafToIcebergType(leaf));
+      case LESS_THAN_EQUALS:
+        return lessThanOrEqual(column, leafToIcebergType(leaf));
+      case IN:
+        return in(column, (List) leafToIcebergType(leaf));
+      case BETWEEN:
+        List<Object> icebergLiterals = leaf.getLiteralList();
+        return and(greaterThanOrEqual(column, icebergLiterals.get(0)),
+                lessThanOrEqual(column, icebergLiterals.get(1)));
+      case IS_NULL:
+        return isNull(column);
+      default:
+        throw new IllegalStateException("Unknown operator: " + leaf.getOperator());
+    }
+  }
+
+  private static Object leafToIcebergType(PredicateLeaf leaf) {

Review comment:
       I think it would be better to split this into two methods: one for a single literal and one for a list of literals. Returning either one as `Object` doesn't allow us to make sure we're calling `getLiteralList` for the correct predicates.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r479703594



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.util.DateTimeUtil;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+
+public class HiveIcebergFilterFactory {
+
+  private HiveIcebergFilterFactory() {
+  }
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(childNodes.get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new UnsupportedOperationException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToLiteral(leaf));
+      case LESS_THAN:
+        return lessThan(column, leafToLiteral(leaf));
+      case LESS_THAN_EQUALS:
+        return lessThanOrEqual(column, leafToLiteral(leaf));
+      case IN:
+        return in(column, leafToLiteralList(leaf));
+      case BETWEEN:
+        List<Object> icebergLiterals = leafToLiteralList(leaf);
+        return and(greaterThanOrEqual(column, icebergLiterals.get(0)),
+                lessThanOrEqual(column, icebergLiterals.get(1)));
+      case IS_NULL:
+        return isNull(column);
+      default:
+        throw new UnsupportedOperationException("Unknown operator: " + leaf.getOperator());
+    }
+  }
+
+  private static Object leafToLiteral(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+      case BOOLEAN:
+      case STRING:
+      case FLOAT:
+        return leaf.getLiteral();
+      case DATE:
+      case TIMESTAMP:
+        // Hive converts a Date type to a Timestamp internally when retrieving literal
+        return timestampToUnixEpoch((Timestamp) leaf.getLiteral());
+      case DECIMAL:
+        return hiveDecimalToBigDecimal((HiveDecimalWritable) leaf.getLiteral());
+
+      default:
+        throw new UnsupportedOperationException("Unknown type: " + leaf.getType());
+    }
+  }
+
+  private static List<Object> leafToLiteralList(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+      case BOOLEAN:
+      case FLOAT:
+      case STRING:
+        return leaf.getLiteralList();
+      case DATE:
+        return leaf.getLiteralList().stream().map(value -> dateToMicros((Date) value))
+                .collect(Collectors.toList());
+      case DECIMAL:
+        return leaf.getLiteralList().stream()
+                .map(value -> hiveDecimalToBigDecimal((HiveDecimalWritable) value))
+                .collect(Collectors.toList());
+      case TIMESTAMP:
+        return leaf.getLiteralList().stream()
+                .map(value -> timestampToUnixEpoch((Timestamp) value))
+                .collect(Collectors.toList());
+      default:
+        throw new UnsupportedOperationException("Unknown type: " + leaf.getType());
+    }
+  }
+
+  private static long dateToMicros(Date date) {
+    return TimeUnit.MILLISECONDS.toMicros(date.toInstant().toEpochMilli());
+  }
+
+  private static BigDecimal hiveDecimalToBigDecimal(HiveDecimalWritable hiveDecimalWritable) {
+    return hiveDecimalWritable.getHiveDecimal().bigDecimalValue().setScale(hiveDecimalWritable.scale());
+  }
+
+  private static long timestampToUnixEpoch(Timestamp timestamp) {

Review comment:
       How about renaming this to `microsFromTimestamp`?
   
   ```java
     private static long microsFromTimestamp(Timestamp timestamp) {
       return DateTimeUtil.microsFromTimestamp(timestamp.toLocalDateTime());
     }
   ```




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdsr commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r473050352



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
##########
@@ -0,0 +1,159 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+
+public class HiveIcebergFilterFactory {
+
+  private static final int MICROS_PER_SECOND = 1000000;
+  private static final int NANOSECS_PER_MICROSEC = 1000;
+
+  private HiveIcebergFilterFactory() {}
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(childNodes.get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new IllegalStateException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToLiteral(leaf));

Review comment:
       yea, that sounds good. 




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdsr commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r472508913



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
##########
@@ -0,0 +1,159 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+
+public class HiveIcebergFilterFactory {
+
+  private static final int MICROS_PER_SECOND = 1000000;
+  private static final int NANOSECS_PER_MICROSEC = 1000;

Review comment:
       nit: to b consistent with `MICROS_PER_SECOND` maybe we can rename `NANOSECS_PER_MICROSEC` to `NANOS_PER_MICROSEC`




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] cmathiesen commented on a change in pull request #1326: Hive: Filter pushdown

Posted by GitBox <gi...@apache.org>.
cmathiesen commented on a change in pull request #1326:
URL: https://github.com/apache/iceberg/pull/1326#discussion_r472931036



##########
File path: api/src/main/java/org/apache/iceberg/ContentFile.java
##########
@@ -102,6 +102,18 @@
    */
   List<Long> splitOffsets();
 
+  /**

Review comment:
       Yeah, trying to sort it out! :L 




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org