You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by om...@apache.org on 2015/07/28 21:49:07 UTC

[4/5] hive git commit: HIVE-10799. Refactor the SearchArgumentFactory to remove the AST-specific factory. (omalley reviewed by prasanth_j)

http://git-wip-us.apache.org/repos/asf/hive/blob/c178a6e9/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestRecordReaderImpl.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestRecordReaderImpl.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestRecordReaderImpl.java
index 957f54e..7957cb4 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestRecordReaderImpl.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestRecordReaderImpl.java
@@ -42,7 +42,9 @@ import org.apache.hadoop.hive.ql.io.orc.RecordReaderImpl.Location;
 import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgument.TruthValue;
 import org.apache.hadoop.hive.ql.io.sarg.TestSearchArgumentImpl;
+import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.junit.Test;
 import org.mockito.MockSettings;
@@ -351,14 +353,14 @@ public class TestRecordReaderImpl {
         RecordReaderImpl.evaluatePredicateProto(createBooleanStats(10, 0), pred, null));
 
     pred = TestSearchArgumentImpl.createPredicateLeaf(
-        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.BOOLEAN, "x", "true", null);
+        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.BOOLEAN, "x", true, null);
     assertEquals(TruthValue.YES_NO,
         RecordReaderImpl.evaluatePredicateProto(createBooleanStats(10, 10), pred, null));
     assertEquals(TruthValue.NO,
         RecordReaderImpl.evaluatePredicateProto(createBooleanStats(10, 0), pred, null));
 
     pred = TestSearchArgumentImpl.createPredicateLeaf(
-        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.BOOLEAN, "x", "hello", null);
+        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.BOOLEAN, "x", false, null);
     assertEquals(TruthValue.NO,
         RecordReaderImpl.evaluatePredicateProto(createBooleanStats(10, 10), pred, null));
     assertEquals(TruthValue.YES_NO,
@@ -368,7 +370,7 @@ public class TestRecordReaderImpl {
   @Test
   public void testPredEvalWithIntStats() throws Exception {
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf(
-        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.INTEGER, "x", 15L, null);
+        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.LONG, "x", 15L, null);
     assertEquals(TruthValue.YES_NO,
         RecordReaderImpl.evaluatePredicateProto(createIntStats(10, 100), pred, null));
 
@@ -390,7 +392,7 @@ public class TestRecordReaderImpl {
         RecordReaderImpl.evaluatePredicateProto(createIntStats(10, 100), pred, null));
 
     pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.DECIMAL, "x", HiveDecimal.create(15), null);
+        PredicateLeaf.Type.DECIMAL, "x", new HiveDecimalWritable("15"), null);
     assertEquals(TruthValue.YES_NO,
         RecordReaderImpl.evaluatePredicateProto(createIntStats(10, 100), pred, null));
 
@@ -403,7 +405,7 @@ public class TestRecordReaderImpl {
   @Test
   public void testPredEvalWithDoubleStats() throws Exception {
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf(
-        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.INTEGER, "x", 15L, null);
+        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.LONG, "x", 15L, null);
     assertEquals(TruthValue.YES_NO,
         RecordReaderImpl.evaluatePredicateProto(createDoubleStats(10.0, 100.0), pred, null));
 
@@ -425,7 +427,7 @@ public class TestRecordReaderImpl {
         RecordReaderImpl.evaluatePredicateProto(createDoubleStats(10.0, 100.0), pred, null));
 
     pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.DECIMAL, "x", HiveDecimal.create(15), null);
+        PredicateLeaf.Type.DECIMAL, "x", new HiveDecimalWritable("15"), null);
     assertEquals(TruthValue.YES_NO,
         RecordReaderImpl.evaluatePredicateProto(createDoubleStats(10.0, 100.0), pred, null));
 
@@ -461,10 +463,10 @@ public class TestRecordReaderImpl {
     pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
         PredicateLeaf.Type.DATE, "x", new DateWritable(100).get(), null);
     assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("10", "1000"), pred, null));
+        RecordReaderImpl.evaluatePredicateProto(createDateStats(10, 1000), pred, null));
 
     pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.DECIMAL, "x", HiveDecimal.create(100), null);
+        PredicateLeaf.Type.DECIMAL, "x", new HiveDecimalWritable("100"), null);
     assertEquals(TruthValue.YES_NO,
         RecordReaderImpl.evaluatePredicateProto(createStringStats("10", "1000"), pred, null));
 
@@ -477,7 +479,7 @@ public class TestRecordReaderImpl {
   @Test
   public void testPredEvalWithDateStats() throws Exception {
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf(
-        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.INTEGER, "x", 15L, null);
+        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.LONG, "x", 15L, null);
     // Date to Integer conversion is not possible.
     assertEquals(TruthValue.YES_NO,
         RecordReaderImpl.evaluatePredicateProto(createDateStats(10, 100), pred, null));
@@ -530,7 +532,7 @@ public class TestRecordReaderImpl {
 
     // Date to Decimal conversion is also not possible.
     pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.DECIMAL, "x", HiveDecimal.create(15), null);
+        PredicateLeaf.Type.DECIMAL, "x", new HiveDecimalWritable("15"), null);
     assertEquals(TruthValue.YES_NO,
         RecordReaderImpl.evaluatePredicateProto(createDateStats(10, 100), pred, null));
 
@@ -548,7 +550,7 @@ public class TestRecordReaderImpl {
   @Test
   public void testPredEvalWithDecimalStats() throws Exception {
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf(
-        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.INTEGER, "x", 15L, null);
+        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.INTEGER, "x", 15, null);
     assertEquals(TruthValue.YES_NO,
         RecordReaderImpl.evaluatePredicateProto(createDecimalStats("10.0", "100.0"), pred, null));
 
@@ -570,7 +572,7 @@ public class TestRecordReaderImpl {
         RecordReaderImpl.evaluatePredicateProto(createDecimalStats("10.0", "100.0"), pred, null));
 
     pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.DECIMAL, "x", HiveDecimal.create(15), null);
+        PredicateLeaf.Type.DECIMAL, "x", new HiveDecimalWritable("15"), null);
     assertEquals(TruthValue.YES_NO,
         RecordReaderImpl.evaluatePredicateProto(createDecimalStats("10.0", "100.0"), pred, null));
 
@@ -588,7 +590,7 @@ public class TestRecordReaderImpl {
   @Test
   public void testPredEvalWithTimestampStats() throws Exception {
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf(
-        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.INTEGER, "x", 15L, null);
+        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.INTEGER, "x", 15, null);
     assertEquals(TruthValue.YES_NO,
         RecordReaderImpl.evaluatePredicateProto(createTimestampStats(10, 100), pred, null));
 
@@ -618,7 +620,7 @@ public class TestRecordReaderImpl {
             100 * 24L * 60L * 60L * 1000L), pred, null));
 
     pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.DECIMAL, "x", HiveDecimal.create(15), null);
+        PredicateLeaf.Type.DECIMAL, "x", new HiveDecimalWritable("15"), null);
     assertEquals(TruthValue.NO,
         RecordReaderImpl.evaluatePredicateProto(createTimestampStats(10, 100), pred, null));
     assertEquals(TruthValue.YES_NO,
@@ -636,7 +638,7 @@ public class TestRecordReaderImpl {
   public void testEquals() throws Exception {
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf
         (PredicateLeaf.Operator.EQUALS, PredicateLeaf.Type.INTEGER,
-            "x", 15L, null);
+            "x", 15, null);
     assertEquals(TruthValue.NO_NULL,
         RecordReaderImpl.evaluatePredicateProto(createIntStats(20L, 30L), pred, null));
     assertEquals(TruthValue.YES_NO_NULL,
@@ -655,7 +657,7 @@ public class TestRecordReaderImpl {
   public void testNullSafeEquals() throws Exception {
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf
         (PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.INTEGER,
-            "x", 15L, null);
+            "x", 15, null);
     assertEquals(TruthValue.NO,
         RecordReaderImpl.evaluatePredicateProto(createIntStats(20L, 30L), pred, null));
     assertEquals(TruthValue.YES_NO,
@@ -674,7 +676,7 @@ public class TestRecordReaderImpl {
   public void testLessThan() throws Exception {
     PredicateLeaf lessThan = TestSearchArgumentImpl.createPredicateLeaf
         (PredicateLeaf.Operator.LESS_THAN, PredicateLeaf.Type.INTEGER,
-            "x", 15L, null);
+            "x", 15, null);
     assertEquals(TruthValue.NO_NULL,
         RecordReaderImpl.evaluatePredicateProto(createIntStats(20L, 30L), lessThan, null));
     assertEquals(TruthValue.NO_NULL,
@@ -691,7 +693,7 @@ public class TestRecordReaderImpl {
   public void testLessThanEquals() throws Exception {
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf
         (PredicateLeaf.Operator.LESS_THAN_EQUALS, PredicateLeaf.Type.INTEGER,
-            "x", 15L, null);
+            "x", 15, null);
     assertEquals(TruthValue.NO_NULL,
         RecordReaderImpl.evaluatePredicateProto(createIntStats(20L, 30L), pred, null));
     assertEquals(TruthValue.YES_NO_NULL,
@@ -707,8 +709,8 @@ public class TestRecordReaderImpl {
   @Test
   public void testIn() throws Exception {
     List<Object> args = new ArrayList<Object>();
-    args.add(10L);
-    args.add(20L);
+    args.add(10);
+    args.add(20);
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf
         (PredicateLeaf.Operator.IN, PredicateLeaf.Type.INTEGER,
             "x", null, args);
@@ -725,8 +727,8 @@ public class TestRecordReaderImpl {
   @Test
   public void testBetween() throws Exception {
     List<Object> args = new ArrayList<Object>();
-    args.add(10L);
-    args.add(20L);
+    args.add(10);
+    args.add(20);
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf
         (PredicateLeaf.Operator.BETWEEN, PredicateLeaf.Type.INTEGER,
             "x", null, args);
@@ -1274,7 +1276,7 @@ public class TestRecordReaderImpl {
   @Test
   public void testIntNullSafeEqualsBloomFilter() throws Exception {
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf(
-        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.INTEGER, "x", 15L, null);
+        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.LONG, "x", 15L, null);
     BloomFilterIO bf = new BloomFilterIO(10000);
     for (int i = 20; i < 1000; i++) {
       bf.addLong(i);
@@ -1289,7 +1291,7 @@ public class TestRecordReaderImpl {
   @Test
   public void testIntEqualsBloomFilter() throws Exception {
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf(
-        PredicateLeaf.Operator.EQUALS, PredicateLeaf.Type.INTEGER, "x", 15L, null);
+        PredicateLeaf.Operator.EQUALS, PredicateLeaf.Type.LONG, "x", 15L, null);
     BloomFilterIO bf = new BloomFilterIO(10000);
     for (int i = 20; i < 1000; i++) {
       bf.addLong(i);
@@ -1539,7 +1541,7 @@ public class TestRecordReaderImpl {
   public void testDecimalNullSafeEqualsBloomFilter() throws Exception {
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf(
         PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.DECIMAL, "x",
-        HiveDecimal.create(15),
+        new HiveDecimalWritable("15"),
         null);
     BloomFilterIO bf = new BloomFilterIO(10000);
     for (int i = 20; i < 1000; i++) {
@@ -1555,7 +1557,8 @@ public class TestRecordReaderImpl {
   @Test
   public void testDecimalEqualsBloomFilter() throws Exception {
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf(
-        PredicateLeaf.Operator.EQUALS, PredicateLeaf.Type.DECIMAL, "x", HiveDecimal.create(15),
+        PredicateLeaf.Operator.EQUALS, PredicateLeaf.Type.DECIMAL, "x",
+        new HiveDecimalWritable("15"),
         null);
     BloomFilterIO bf = new BloomFilterIO(10000);
     for (int i = 20; i < 1000; i++) {
@@ -1571,8 +1574,8 @@ public class TestRecordReaderImpl {
   @Test
   public void testDecimalInBloomFilter() throws Exception {
     List<Object> args = new ArrayList<Object>();
-    args.add(HiveDecimal.create(15));
-    args.add(HiveDecimal.create(19));
+    args.add(new HiveDecimalWritable("15"));
+    args.add(new HiveDecimalWritable("19"));
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf
         (PredicateLeaf.Operator.IN, PredicateLeaf.Type.DECIMAL,
             "x", null, args);
@@ -1593,9 +1596,9 @@ public class TestRecordReaderImpl {
   @Test
   public void testNullsInBloomFilter() throws Exception {
     List<Object> args = new ArrayList<Object>();
-    args.add(HiveDecimal.create(15));
+    args.add(new HiveDecimalWritable("15"));
     args.add(null);
-    args.add(HiveDecimal.create(19));
+    args.add(new HiveDecimalWritable("19"));
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf
         (PredicateLeaf.Operator.IN, PredicateLeaf.Type.DECIMAL,
             "x", null, args);

http://git-wip-us.apache.org/repos/asf/hive/blob/c178a6e9/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestParquetRecordReaderWrapper.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestParquetRecordReaderWrapper.java b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestParquetRecordReaderWrapper.java
new file mode 100644
index 0000000..87dd344
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestParquetRecordReaderWrapper.java
@@ -0,0 +1,155 @@
+/**
+ * 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.hadoop.hive.ql.io.parquet;
+
+import static junit.framework.Assert.assertEquals;
+
+import org.apache.hadoop.hive.common.type.HiveChar;
+import org.apache.hadoop.hive.common.type.HiveVarchar;
+import org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper;
+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.SearchArgument.TruthValue;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentFactory;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.junit.Test;
+
+import java.sql.Date;
+
+import org.apache.parquet.filter2.predicate.FilterPredicate;
+
+/**
+ * These tests test the conversion to Parquet's sarg implementation.
+ */
+public class TestParquetRecordReaderWrapper {
+
+  private static TruthValue[] values(TruthValue... vals) {
+    return vals;
+  }
+
+  @Test
+  public void testBuilder() throws Exception {
+     SearchArgument sarg = SearchArgumentFactory.newBuilder()
+        .startNot()
+        .startOr()
+        .isNull("x", PredicateLeaf.Type.INTEGER)
+        .between("y", PredicateLeaf.Type.INTEGER, 10, 20)
+        .in("z", PredicateLeaf.Type.INTEGER, 1, 2, 3)
+        .nullSafeEquals("a", PredicateLeaf.Type.STRING, "stinger")
+        .end()
+        .end()
+        .build();
+
+    FilterPredicate p = ParquetRecordReaderWrapper.toFilterPredicate(sarg);
+    String expected =
+      "and(and(and(not(eq(x, null)), not(and(lt(y, 20), not(lteq(y, 10))))), not(or(or(eq(z, 1), " +
+        "eq(z, 2)), eq(z, 3)))), not(eq(a, Binary{\"stinger\"})))";
+    assertEquals(expected, p.toString());
+  }
+
+  @Test
+  public void testBuilderComplexTypes() throws Exception {
+    SearchArgument sarg =
+        SearchArgumentFactory.newBuilder()
+            .startAnd()
+            .lessThan("x", PredicateLeaf.Type.DATE,
+                Date.valueOf("1970-1-11"))
+            .lessThanEquals("y", PredicateLeaf.Type.STRING,
+                new HiveChar("hi", 10).toString())
+            .equals("z", PredicateLeaf.Type.DECIMAL, new HiveDecimalWritable("1.0"))
+            .end()
+            .build();
+    assertEquals("lteq(y, Binary{\"hi        \"})",
+        ParquetRecordReaderWrapper.toFilterPredicate(sarg).toString());
+
+    sarg = SearchArgumentFactory.newBuilder()
+        .startNot()
+        .startOr()
+        .isNull("x", PredicateLeaf.Type.INTEGER)
+        .between("y", PredicateLeaf.Type.DECIMAL,
+            new HiveDecimalWritable("10"), new HiveDecimalWritable("20.0"))
+        .in("z", PredicateLeaf.Type.INTEGER, 1, 2, 3)
+        .nullSafeEquals("a", PredicateLeaf.Type.STRING,
+            new HiveVarchar("stinger", 100).toString())
+        .end()
+        .end()
+        .build();
+
+    FilterPredicate p = ParquetRecordReaderWrapper.toFilterPredicate(sarg);
+    String expected =
+        "and(and(not(eq(x, null)), not(or(or(eq(z, 1), eq(z, 2)), eq(z, 3)))), " +
+        "not(eq(a, Binary{\"stinger\"})))";
+    assertEquals(expected, p.toString());
+  }
+
+  @Test
+  public void testBuilderComplexTypes2() throws Exception {
+    SearchArgument sarg =
+        SearchArgumentFactory.newBuilder()
+            .startAnd()
+            .lessThan("x", PredicateLeaf.Type.DATE, Date.valueOf("2005-3-12"))
+            .lessThanEquals("y", PredicateLeaf.Type.STRING,
+                new HiveChar("hi", 10).toString())
+            .equals("z", PredicateLeaf.Type.DECIMAL,
+                new HiveDecimalWritable("1.0"))
+            .end()
+            .build();
+    assertEquals("lteq(y, Binary{\"hi        \"})",
+        ParquetRecordReaderWrapper.toFilterPredicate(sarg).toString());
+
+    sarg = SearchArgumentFactory.newBuilder()
+        .startNot()
+        .startOr()
+        .isNull("x", PredicateLeaf.Type.INTEGER)
+        .between("y", PredicateLeaf.Type.DECIMAL, new HiveDecimalWritable("10"),
+            new HiveDecimalWritable("20.0"))
+        .in("z", PredicateLeaf.Type.INTEGER, 1, 2, 3)
+        .nullSafeEquals("a", PredicateLeaf.Type.STRING,
+            new HiveVarchar("stinger", 100).toString())
+        .end()
+        .end()
+        .build();
+
+    FilterPredicate p = ParquetRecordReaderWrapper.toFilterPredicate(sarg);
+    String expected = "and(and(not(eq(x, null)), not(or(or(eq(z, 1), eq(z, 2)), eq(z, 3)))), " +
+        "not(eq(a, Binary{\"stinger\"})))";
+    assertEquals(expected, p.toString());
+  }
+
+  @Test
+  public void testBuilderFloat() throws Exception {
+    SearchArgument sarg =
+        SearchArgumentFactory.newBuilder()
+            .startAnd()
+            .lessThan("x", PredicateLeaf.Type.INTEGER, new Integer((short) 22))
+            .lessThan("x1", PredicateLeaf.Type.INTEGER, new Integer(22))
+            .lessThanEquals("y", PredicateLeaf.Type.STRING,
+                new HiveChar("hi", 10).toString())
+            .equals("z", PredicateLeaf.Type.FLOAT, new Double(0.22))
+            .equals("z1", PredicateLeaf.Type.FLOAT, new Double(0.22))
+            .end()
+            .build();
+
+    FilterPredicate p = ParquetRecordReaderWrapper.toFilterPredicate(sarg);
+    String expected = "and(and(and(and(lt(x, 22), lt(x1, 22))," +
+        " lteq(y, Binary{\"hi        \"})), eq(z, " +
+        "0.22)), eq(z1, 0.22))";
+    assertEquals(expected, p.toString());
+  }
+}