Return-Path: X-Original-To: apmail-hive-commits-archive@www.apache.org Delivered-To: apmail-hive-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id A105D18112 for ; Fri, 31 Jul 2015 00:43:06 +0000 (UTC) Received: (qmail 10135 invoked by uid 500); 31 Jul 2015 00:43:02 -0000 Delivered-To: apmail-hive-commits-archive@hive.apache.org Received: (qmail 10012 invoked by uid 500); 31 Jul 2015 00:43:02 -0000 Mailing-List: contact commits-help@hive.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: hive-dev@hive.apache.org Delivered-To: mailing list commits@hive.apache.org Received: (qmail 8700 invoked by uid 99); 31 Jul 2015 00:43:01 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 31 Jul 2015 00:43:01 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 8F455E7156; Fri, 31 Jul 2015 00:43:01 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: xuefu@apache.org To: commits@hive.apache.org Date: Fri, 31 Jul 2015 00:43:29 -0000 Message-Id: <0284a48747434606b2d9c94a01d4efef@git.apache.org> In-Reply-To: References: X-Mailer: ASF-Git Admin Mailer Subject: [29/43] 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 args = new ArrayList(); - 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 args = new ArrayList(); - 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 args = new ArrayList(); - 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 args = new ArrayList(); - 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()); + } +}