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:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to