You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by pr...@apache.org on 2023/10/06 15:11:13 UTC
[hive] branch branch-3 updated: HIVE-27584: Backport of HIVE-21407: Parquet predicate pushdown is not working correctly for char column types (Marta Kuczora reviewed by Peter Vary)
This is an automated email from the ASF dual-hosted git repository.
pravin pushed a commit to branch branch-3
in repository https://gitbox.apache.org/repos/asf/hive.git
The following commit(s) were added to refs/heads/branch-3 by this push:
new f40d7842829 HIVE-27584: Backport of HIVE-21407: Parquet predicate pushdown is not working correctly for char column types (Marta Kuczora reviewed by Peter Vary)
f40d7842829 is described below
commit f40d784282975ce3be250dcd7371705a24ca1eba
Author: Pravin Kumar Sinha <pr...@apache.org>
AuthorDate: Fri Oct 6 20:41:07 2023 +0530
HIVE-27584: Backport of HIVE-21407: Parquet predicate pushdown is not working correctly for char column types (Marta Kuczora reviewed by Peter Vary)
Signed-off-by: Chinna Rao L <ch...@apache.org>
Closes (#4567)
---
.../ql/io/parquet/FilterPredicateLeafBuilder.java | 15 +-
.../hive/ql/io/parquet/LeafFilterFactory.java | 41 +-
.../ql/io/parquet/ParquetRecordReaderBase.java | 16 +-
.../read/ParquetFilterPredicateConverter.java | 29 +-
.../parquet/read/ParquetRecordReaderWrapper.java | 2 +-
.../io/parquet/TestParquetRecordReaderWrapper.java | 170 -------
.../parquet/read/TestParquetFilterPredicate.java | 564 ++++++++++++++++++++-
.../hive/ql/io/sarg/TestConvertAstToSearchArg.java | 59 ++-
.../queries/clientpositive/parquet_ppd_char2.q | 119 +++++
.../results/clientpositive/parquet_ppd_char2.q.out | 551 ++++++++++++++++++++
10 files changed, 1348 insertions(+), 218 deletions(-)
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/FilterPredicateLeafBuilder.java b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/FilterPredicateLeafBuilder.java
index 5b051dd9ad9..46e45b2ec77 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/FilterPredicateLeafBuilder.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/FilterPredicateLeafBuilder.java
@@ -16,6 +16,7 @@ package org.apache.hadoop.hive.ql.io.parquet;
import java.util.List;
import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
import org.apache.parquet.filter2.predicate.FilterApi;
import org.apache.parquet.filter2.predicate.FilterPredicate;
@@ -33,19 +34,20 @@ public abstract class FilterPredicateLeafBuilder {
* @param op IN or BETWEEN
* @param literals
* @param columnName
+ * @param columnType
* @return
*/
public FilterPredicate buildPredicate(PredicateLeaf.Operator op, List<Object> literals,
- String columnName) throws Exception {
+ String columnName, TypeInfo columnType) throws Exception {
FilterPredicate result = null;
switch (op) {
case IN:
for (Object literal : literals) {
if (result == null) {
- result = buildPredict(PredicateLeaf.Operator.EQUALS, literal, columnName);
+ result = buildPredict(PredicateLeaf.Operator.EQUALS, literal, columnName, columnType);
} else {
result = or(result, buildPredict(PredicateLeaf.Operator.EQUALS, literal,
- columnName));
+ columnName, columnType));
}
}
return result;
@@ -58,8 +60,8 @@ public abstract class FilterPredicateLeafBuilder {
Object min = literals.get(0);
Object max = literals.get(1);
FilterPredicate lt = not(buildPredict(PredicateLeaf.Operator.LESS_THAN,
- min, columnName));
- FilterPredicate gt = buildPredict(PredicateLeaf.Operator.LESS_THAN_EQUALS, max, columnName);
+ min, columnName, columnType));
+ FilterPredicate gt = buildPredict(PredicateLeaf.Operator.LESS_THAN_EQUALS, max, columnName, columnType);
result = FilterApi.and(gt, lt);
return result;
default:
@@ -73,8 +75,9 @@ public abstract class FilterPredicateLeafBuilder {
* @param op EQUALS, NULL_SAFE_EQUALS, LESS_THAN, LESS_THAN_EQUALS, IS_NULL
* @param constant
* @param columnName
+ * @param columnType
* @return null or a FilterPredicate, null means no filter will be executed
*/
public abstract FilterPredicate buildPredict(PredicateLeaf.Operator op, Object constant,
- String columnName) throws Exception;
+ String columnName, TypeInfo columnType) throws Exception;
}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/LeafFilterFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/LeafFilterFactory.java
index be4c0d55a11..965411a7f8a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/LeafFilterFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/LeafFilterFactory.java
@@ -19,6 +19,8 @@ import org.slf4j.LoggerFactory;
import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf.Operator;
import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.serde.serdeConstants;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
import org.apache.parquet.filter2.predicate.FilterApi;
import org.apache.parquet.filter2.predicate.FilterPredicate;
import org.apache.parquet.io.api.Binary;
@@ -42,11 +44,12 @@ public class LeafFilterFactory {
* @param op consists of EQUALS, NULL_SAFE_EQUALS, LESS_THAN, LESS_THAN_EQUALS, IS_NULL
* @param literal
* @param columnName
+ * @param columnType
* @return
*/
@Override
public FilterPredicate buildPredict(Operator op, Object literal,
- String columnName) {
+ String columnName, TypeInfo columnType) {
switch (op) {
case LESS_THAN:
return lt(intColumn(columnName), ((Number) literal).intValue());
@@ -66,7 +69,7 @@ public class LeafFilterFactory {
class LongFilterPredicateLeafBuilder extends FilterPredicateLeafBuilder {
@Override
public FilterPredicate buildPredict(Operator op, Object constant,
- String columnName) {
+ String columnName, TypeInfo columnType) {
switch (op) {
case LESS_THAN:
return lt(FilterApi.longColumn(columnName), ((Number) constant).longValue());
@@ -86,7 +89,7 @@ public class LeafFilterFactory {
class FloatFilterPredicateLeafBuilder extends FilterPredicateLeafBuilder {
@Override
- public FilterPredicate buildPredict(Operator op, Object constant, String columnName) {
+ public FilterPredicate buildPredict(Operator op, Object constant, String columnName, TypeInfo columnType) {
switch (op) {
case LESS_THAN:
return lt(floatColumn(columnName), ((Number) constant).floatValue());
@@ -107,7 +110,7 @@ public class LeafFilterFactory {
@Override
public FilterPredicate buildPredict(Operator op, Object constant,
- String columnName) {
+ String columnName, TypeInfo columnType) {
switch (op) {
case LESS_THAN:
return lt(doubleColumn(columnName), ((Number) constant).doubleValue());
@@ -128,7 +131,7 @@ public class LeafFilterFactory {
class BooleanFilterPredicateLeafBuilder extends FilterPredicateLeafBuilder {
@Override
public FilterPredicate buildPredict(Operator op, Object constant,
- String columnName) throws Exception{
+ String columnName, TypeInfo columnType) throws Exception{
switch (op) {
case IS_NULL:
case EQUALS:
@@ -144,22 +147,42 @@ public class LeafFilterFactory {
class BinaryFilterPredicateLeafBuilder extends FilterPredicateLeafBuilder {
@Override
public FilterPredicate buildPredict(Operator op, Object constant,
- String columnName) throws Exception{
+ String columnName, TypeInfo columnType) throws Exception{
+ // For CHAR types, the trailing spaces should be removed before adding the
+ // value to the predicate. This change is needed because for CHAR types,
+ // Hive passes a padded value to the predicate, but since the value
+ // is stored in Parquet without padding, no result would be returned.
+ // For more details about this issue, please refer to HIVE-21407.
+ String value = null;
+ if (constant != null) {
+ value = (String) constant;
+ if (columnType != null && columnType.toString().startsWith(serdeConstants.CHAR_TYPE_NAME)) {
+ value = removeTrailingSpaces(value);
+ }
+ }
switch (op) {
case LESS_THAN:
- return lt(binaryColumn(columnName), Binary.fromString((String) constant));
+ return lt(binaryColumn(columnName), Binary.fromString(value));
case IS_NULL:
case EQUALS:
case NULL_SAFE_EQUALS:
return eq(binaryColumn(columnName),
- (constant == null) ? null : Binary.fromString((String) constant));
+ (constant == null) ? null : Binary.fromString(value));
case LESS_THAN_EQUALS:
- return ltEq(binaryColumn(columnName), Binary.fromString((String) constant));
+ return ltEq(binaryColumn(columnName), Binary.fromString(value));
default:
// should never be executed
throw new RuntimeException("Unknown PredicateLeaf Operator type: " + op);
}
}
+
+ private String removeTrailingSpaces(String value) {
+ if (value == null) {
+ return null;
+ }
+ String regex = "\\s+$";
+ return value.replaceAll(regex, "");
+ }
}
/**
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/ParquetRecordReaderBase.java b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/ParquetRecordReaderBase.java
index 033e26a2388..91a02feb209 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/ParquetRecordReaderBase.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/ParquetRecordReaderBase.java
@@ -16,11 +16,14 @@ package org.apache.hadoop.hive.ql.io.parquet;
import com.google.common.base.Strings;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.io.IOConstants;
import org.apache.hadoop.hive.ql.io.parquet.read.DataWritableReadSupport;
import org.apache.hadoop.hive.ql.io.parquet.read.ParquetFilterPredicateConverter;
import org.apache.hadoop.hive.ql.io.sarg.ConvertAstToSearchArg;
import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
import org.apache.hadoop.hive.serde2.SerDeStats;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
import org.apache.hadoop.mapred.FileSplit;
import org.apache.hadoop.mapred.JobConf;
import org.apache.parquet.filter2.compat.FilterCompat;
@@ -41,7 +44,9 @@ import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.ArrayList;
+import java.util.HashMap;
import java.util.List;
+import java.util.Map;
public class ParquetRecordReaderBase {
public static final Logger LOG = LoggerFactory.getLogger(ParquetRecordReaderBase.class);
@@ -146,9 +151,18 @@ public class ParquetRecordReaderBase {
return null;
}
+ String columnTypes = conf.get(IOConstants.COLUMNS_TYPES);
+ String columnNames = conf.get(IOConstants.COLUMNS);
+ List<TypeInfo> columnTypeList = TypeInfoUtils.getTypeInfosFromTypeString(columnTypes);
+ Map<String, TypeInfo> columns = new HashMap<>();
+ String[] names = columnNames.split(",");
+ for (int i = 0; i < names.length; i++) {
+ columns.put(names[i], columnTypeList.get(i));
+ }
+
// Create the Parquet FilterPredicate without including columns that do not exist
// on the schema (such as partition columns).
- FilterPredicate p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema);
+ FilterPredicate p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema, columns);
if (p != null) {
// Filter may have sensitive information. Do not send to debug.
LOG.debug("PARQUET predicate push down generated.");
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/ParquetFilterPredicateConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/ParquetFilterPredicateConverter.java
index ca5e085b856..05cb42ce14a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/ParquetFilterPredicateConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/ParquetFilterPredicateConverter.java
@@ -24,14 +24,15 @@ import org.apache.hadoop.hive.ql.io.parquet.LeafFilterFactory;
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.typeinfo.TypeInfo;
import org.apache.parquet.filter2.predicate.FilterApi;
import org.apache.parquet.filter2.predicate.FilterPredicate;
import org.apache.parquet.schema.MessageType;
import org.apache.parquet.schema.Type;
-import java.util.HashSet;
+import java.util.HashMap;
import java.util.List;
-import java.util.Set;
+import java.util.Map;
public class ParquetFilterPredicateConverter {
private static final Logger LOG = LoggerFactory.getLogger(ParquetFilterPredicateConverter.class);
@@ -42,12 +43,13 @@ public class ParquetFilterPredicateConverter {
* @return a filter predicate translated from search argument. null is returned
* if failed to convert.
*/
- public static FilterPredicate toFilterPredicate(SearchArgument sarg, MessageType schema) {
- Set<String> columns = null;
+ public static FilterPredicate toFilterPredicate(SearchArgument sarg, MessageType schema, Map<String, TypeInfo> columnTypes) {
+ Map<String, TypeInfo> columns = null;
if (schema != null) {
- columns = new HashSet<String>();
+ columns = new HashMap<>();
for (Type field : schema.getFields()) {
- columns.add(field.getName());
+ String columnName = field.getName();
+ columns.put(columnName, columnTypes.get(columnName));
}
}
@@ -60,7 +62,7 @@ public class ParquetFilterPredicateConverter {
private static FilterPredicate translate(ExpressionTree root,
List<PredicateLeaf> leaves,
- Set<String> columns,
+ Map<String, TypeInfo> columns,
MessageType schema) throws Exception {
FilterPredicate p = null;
switch (root.getOperator()) {
@@ -103,9 +105,10 @@ public class ParquetFilterPredicateConverter {
PredicateLeaf leaf = leaves.get(root.getLeaf());
// If columns is null, then we need to create the leaf
- if (columns.contains(leaf.getColumnName())) {
+ if (columns.containsKey(leaf.getColumnName())) {
Type parquetType = schema.getType(leaf.getColumnName());
- return buildFilterPredicateFromPredicateLeaf(leaf, parquetType);
+ TypeInfo hiveType = columns.get(leaf.getColumnName());
+ return buildFilterPredicateFromPredicateLeaf(leaf, parquetType, hiveType);
} else {
// Do not create predicate if the leaf is not on the passed schema.
return null;
@@ -119,7 +122,7 @@ public class ParquetFilterPredicateConverter {
}
private static FilterPredicate buildFilterPredicateFromPredicateLeaf
- (PredicateLeaf leaf, Type parquetType) throws Exception {
+ (PredicateLeaf leaf, Type parquetType, TypeInfo columnType) throws Exception {
LeafFilterFactory leafFilterFactory = new LeafFilterFactory();
FilterPredicateLeafBuilder builder;
try {
@@ -129,12 +132,14 @@ public class ParquetFilterPredicateConverter {
if (isMultiLiteralsOperator(leaf.getOperator())) {
return builder.buildPredicate(leaf.getOperator(),
leaf.getLiteralList(),
- leaf.getColumnName());
+ leaf.getColumnName(),
+ columnType);
} else {
return builder
.buildPredict(leaf.getOperator(),
leaf.getLiteral(),
- leaf.getColumnName());
+ leaf.getColumnName(),
+ columnType);
}
} catch (Exception e) {
LOG.error("fail to build predicate filter leaf with errors" + e, e);
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/ParquetRecordReaderWrapper.java b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/ParquetRecordReaderWrapper.java
index 3e0c7cbc109..113d61f5f97 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/ParquetRecordReaderWrapper.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/ParquetRecordReaderWrapper.java
@@ -83,7 +83,7 @@ public class ParquetRecordReaderWrapper extends ParquetRecordReaderBase
Configuration conf = jobConf;
if (skipTimestampConversion ^ HiveConf.getBoolVar(
conf, HiveConf.ConfVars.HIVE_PARQUET_TIMESTAMP_SKIP_CONVERSION)) {
- conf = new JobConf(oldJobConf);
+ conf = new JobConf(jobConf);
HiveConf.setBoolVar(conf,
HiveConf.ConfVars.HIVE_PARQUET_TIMESTAMP_SKIP_CONVERSION, skipTimestampConversion);
}
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
deleted file mode 100644
index c4a4c21cfe0..00000000000
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestParquetRecordReaderWrapper.java
+++ /dev/null
@@ -1,170 +0,0 @@
-/*
- * 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 org.apache.hadoop.hive.common.type.HiveChar;
-import org.apache.hadoop.hive.common.type.HiveVarchar;
-import org.apache.hadoop.hive.ql.io.parquet.read.ParquetFilterPredicateConverter;
-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.parquet.schema.MessageType;
-import org.apache.parquet.schema.MessageTypeParser;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-
-import java.sql.Date;
-
-import org.apache.parquet.filter2.predicate.FilterPredicate;
-
-/**
- * These tests test the conversion to Parquet's sarg implementation.
- */
-public class TestParquetRecordReaderWrapper {
-
- @Test
- public void testBuilder() throws Exception {
- SearchArgument sarg = SearchArgumentFactory.newBuilder()
- .startNot()
- .startOr()
- .isNull("x", PredicateLeaf.Type.LONG)
- .between("y", PredicateLeaf.Type.LONG, 10L, 20L)
- .in("z", PredicateLeaf.Type.LONG, 1L, 2L, 3L)
- .nullSafeEquals("a", PredicateLeaf.Type.STRING, "stinger")
- .end()
- .end()
- .build();
-
- MessageType schema = MessageTypeParser.parseMessageType("message test {" +
- " optional int32 x; required int32 y; required int32 z;" +
- " optional binary a;}");
- FilterPredicate p =
- ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema);
- String expected =
- "and(and(and(not(eq(x, null)), not(and(lteq(y, 20), not(lt(y, 10))))), not(or(or(eq(z, 1), " +
- "eq(z, 2)), eq(z, 3)))), not(eq(a, Binary{\"stinger\"})))";
- assertEquals(expected, p.toString());
- }
-
- /**
- * Check the converted filter predicate is null if unsupported types are included
- * @throws Exception
- */
- @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();
- MessageType schema = MessageTypeParser.parseMessageType("message test {" +
- " required int32 x; required binary y; required binary z;}");
- assertEquals(null,
- ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema));
-
- sarg = SearchArgumentFactory.newBuilder()
- .startNot()
- .startOr()
- .isNull("x", PredicateLeaf.Type.LONG)
- .between("y", PredicateLeaf.Type.DECIMAL,
- new HiveDecimalWritable("10"), new HiveDecimalWritable("20.0"))
- .in("z", PredicateLeaf.Type.LONG, 1L, 2L, 3L)
- .nullSafeEquals("a", PredicateLeaf.Type.STRING,
- new HiveVarchar("stinger", 100).toString())
- .end()
- .end()
- .build();
- schema = MessageTypeParser.parseMessageType("message test {" +
- " optional int32 x; required binary y; required int32 z;" +
- " optional binary a;}");
- assertEquals(null,
- ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema));
- }
-
- /**
- * Check the converted filter predicate is null if unsupported types are included
- * @throws Exception
- */
- @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();
- MessageType schema = MessageTypeParser.parseMessageType("message test {" +
- " required int32 x; required binary y; required binary z;}");
- assertEquals(null,
- ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema));
-
- sarg = SearchArgumentFactory.newBuilder()
- .startNot()
- .startOr()
- .isNull("x", PredicateLeaf.Type.LONG)
- .between("y", PredicateLeaf.Type.DECIMAL, new HiveDecimalWritable("10"),
- new HiveDecimalWritable("20.0"))
- .in("z", PredicateLeaf.Type.LONG, 1L, 2L, 3L)
- .nullSafeEquals("a", PredicateLeaf.Type.STRING,
- new HiveVarchar("stinger", 100).toString())
- .end()
- .end()
- .build();
- schema = MessageTypeParser.parseMessageType("message test {" +
- " optional int32 x; required binary y; required int32 z;" +
- " optional binary a;}");
- assertEquals(null,
- ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema));
- }
-
- @Test
- public void testBuilderFloat() throws Exception {
- SearchArgument sarg =
- SearchArgumentFactory.newBuilder()
- .startAnd()
- .lessThan("x", PredicateLeaf.Type.LONG, 22L)
- .lessThan("x1", PredicateLeaf.Type.LONG, 22L)
- .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();
- MessageType schema = MessageTypeParser.parseMessageType("message test {" +
- " required int32 x; required int32 x1;" +
- " required binary y; required float z; required float z1;}");
-
- FilterPredicate p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema);
- 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());
- }
-}
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/read/TestParquetFilterPredicate.java b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/read/TestParquetFilterPredicate.java
index d46404648d4..a26f0140046 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/read/TestParquetFilterPredicate.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/read/TestParquetFilterPredicate.java
@@ -18,15 +18,24 @@
package org.apache.hadoop.hive.ql.io.parquet.read;
+import static org.junit.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.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.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
import org.apache.parquet.filter2.predicate.FilterPredicate;
import org.apache.parquet.schema.MessageType;
import org.apache.parquet.schema.MessageTypeParser;
import org.junit.Test;
-import static junit.framework.Assert.assertEquals;
+import java.sql.Date;
+import java.util.HashMap;
+import java.util.Map;
public class TestParquetFilterPredicate {
@Test
@@ -43,7 +52,13 @@ public class TestParquetFilterPredicate {
.end()
.build();
- FilterPredicate p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema);
+ Map<String, TypeInfo> columnTypes = new HashMap<>();
+ columnTypes.put("a", TypeInfoFactory.getPrimitiveTypeInfo("int"));
+ columnTypes.put("y", TypeInfoFactory.getPrimitiveTypeInfo("int"));
+ columnTypes.put("z", TypeInfoFactory.getPrimitiveTypeInfo("int"));
+ columnTypes.put("stinger", TypeInfoFactory.getPrimitiveTypeInfo("string"));
+
+ FilterPredicate p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema, columnTypes);
String expected = "and(not(eq(a, null)), not(eq(a, Binary{\"stinger\"})))";
assertEquals(expected, p.toString());
@@ -65,7 +80,11 @@ public class TestParquetFilterPredicate {
.end()
.build();
- FilterPredicate p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema);
+ Map<String, TypeInfo> columnTypes = new HashMap<>();
+ columnTypes.put("a", TypeInfoFactory.getPrimitiveTypeInfo("int"));
+ columnTypes.put("none", TypeInfoFactory.getPrimitiveTypeInfo("int"));
+
+ FilterPredicate p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema, columnTypes);
String expected = "or(eq(a, 1), eq(a, 999))";
assertEquals(expected, p.toString());
@@ -87,7 +106,12 @@ public class TestParquetFilterPredicate {
.end()
.build();
- FilterPredicate p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema);
+ Map<String, TypeInfo> columnTypes = new HashMap<>();
+ columnTypes.put("a", TypeInfoFactory.getPrimitiveTypeInfo("int"));
+ columnTypes.put("b", TypeInfoFactory.getPrimitiveTypeInfo("int"));
+ columnTypes.put("none", TypeInfoFactory.getPrimitiveTypeInfo("int"));
+
+ FilterPredicate p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema, columnTypes);
String expected = "or(eq(a, 1), eq(b, 1))";
assertEquals(expected, p.toString());
@@ -107,7 +131,11 @@ public class TestParquetFilterPredicate {
.end()
.build();
- FilterPredicate p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema);
+ Map<String, TypeInfo> columnTypes = new HashMap<>();
+ columnTypes.put("a", TypeInfoFactory.getPrimitiveTypeInfo("float"));
+ columnTypes.put("b", TypeInfoFactory.getPrimitiveTypeInfo("int"));
+
+ FilterPredicate p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema, columnTypes);
String expected =
"and(and(not(eq(a, null)), not(and(lteq(a, 20.3), not(lt(a, 10.2))))), not(or(or(eq(b, 1), eq(b, 2)), eq(b, 3))))";
@@ -121,7 +149,9 @@ public class TestParquetFilterPredicate {
SearchArgument sarg = SearchArgumentFactory.newBuilder()
.between("bCol", PredicateLeaf.Type.LONG, 1L, 5L)
.build();
- FilterPredicate p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema);
+ Map<String, TypeInfo> columnTypes = new HashMap<>();
+ columnTypes.put("bCol", TypeInfoFactory.getPrimitiveTypeInfo("int"));
+ FilterPredicate p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema, columnTypes);
String expected =
"and(lteq(bCol, 5), not(lt(bCol, 1)))";
assertEquals(expected, p.toString());
@@ -129,7 +159,7 @@ public class TestParquetFilterPredicate {
sarg = SearchArgumentFactory.newBuilder()
.between("bCol", PredicateLeaf.Type.LONG, 5L, 1L)
.build();
- p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema);
+ p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema, columnTypes);
expected =
"and(lteq(bCol, 1), not(lt(bCol, 5)))";
assertEquals(expected, p.toString());
@@ -137,9 +167,527 @@ public class TestParquetFilterPredicate {
sarg = SearchArgumentFactory.newBuilder()
.between("bCol", PredicateLeaf.Type.LONG, 1L, 1L)
.build();
- p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema);
+ p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema, columnTypes);
expected =
"and(lteq(bCol, 1), not(lt(bCol, 1)))";
assertEquals(expected, p.toString());
}
+
+ @Test
+ public void testFilter() throws Exception {
+ SearchArgument sarg = SearchArgumentFactory.newBuilder()
+ .startNot()
+ .startOr()
+ .isNull("x", PredicateLeaf.Type.LONG)
+ .between("y", PredicateLeaf.Type.LONG, 10L, 20L)
+ .in("z", PredicateLeaf.Type.LONG, 1L, 2L, 3L)
+ .nullSafeEquals("a", PredicateLeaf.Type.STRING, "stinger")
+ .end()
+ .end()
+ .build();
+
+ MessageType schema = MessageTypeParser.parseMessageType("message test {" +
+ " optional int32 x; required int32 y; required int32 z;" +
+ " optional binary a;}");
+ Map<String, TypeInfo> columnTypes = new HashMap<>();
+ columnTypes.put("x", TypeInfoFactory.getPrimitiveTypeInfo("int"));
+ columnTypes.put("y", TypeInfoFactory.getPrimitiveTypeInfo("int"));
+ columnTypes.put("z", TypeInfoFactory.getPrimitiveTypeInfo("int"));
+ columnTypes.put("a", TypeInfoFactory.getPrimitiveTypeInfo("string"));
+ FilterPredicate p =
+ ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema, columnTypes);
+ String expected =
+ "and(and(and(not(eq(x, null)), not(and(lteq(y, 20), not(lt(y, 10))))), not(or(or(eq(z, 1), " +
+ "eq(z, 2)), eq(z, 3)))), not(eq(a, Binary{\"stinger\"})))";
+ assertEquals(expected, p.toString());
+ }
+
+ /**
+ * Check the converted filter predicate is null if unsupported types are included
+ * @throws Exception
+ */
+ @Test
+ public void testFilterComplexTypes() 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();
+ MessageType schema = MessageTypeParser.parseMessageType("message test {" +
+ " required int32 x; required binary y; required binary z;}");
+ Map<String, TypeInfo> columnTypes = new HashMap<>();
+ columnTypes.put("x", TypeInfoFactory.getPrimitiveTypeInfo("date"));
+ columnTypes.put("y", TypeInfoFactory.getCharTypeInfo(10));
+ columnTypes.put("z", TypeInfoFactory.getDecimalTypeInfo(4, 2));
+ assertEquals(null,
+ ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema, columnTypes));
+
+ sarg = SearchArgumentFactory.newBuilder()
+ .startNot()
+ .startOr()
+ .isNull("x", PredicateLeaf.Type.LONG)
+ .between("y", PredicateLeaf.Type.DECIMAL,
+ new HiveDecimalWritable("10"), new HiveDecimalWritable("20.0"))
+ .in("z", PredicateLeaf.Type.LONG, 1L, 2L, 3L)
+ .nullSafeEquals("a", PredicateLeaf.Type.STRING,
+ new HiveVarchar("stinger", 100).toString())
+ .end()
+ .end()
+ .build();
+ schema = MessageTypeParser.parseMessageType("message test {" +
+ " optional int32 x; required binary y; required int32 z;" +
+ " optional binary a;}");
+ columnTypes = new HashMap<>();
+ columnTypes.put("x", TypeInfoFactory.getPrimitiveTypeInfo("int"));
+ columnTypes.put("y", TypeInfoFactory.getDecimalTypeInfo(4, 2));
+ columnTypes.put("z", TypeInfoFactory.getPrimitiveTypeInfo("int"));
+ columnTypes.put("z", TypeInfoFactory.getCharTypeInfo(100));
+ assertEquals(null,
+ ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema, columnTypes));
+ }
+
+ /**
+ * Check the converted filter predicate is null if unsupported types are included
+ * @throws Exception
+ */
+ @Test
+ public void testFilterComplexTypes2() 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();
+ MessageType schema = MessageTypeParser.parseMessageType("message test {" +
+ " required int32 x; required binary y; required binary z;}");
+ Map<String, TypeInfo> columnTypes = new HashMap<>();
+ columnTypes.put("x", TypeInfoFactory.getPrimitiveTypeInfo("date"));
+ columnTypes.put("y", TypeInfoFactory.getCharTypeInfo(10));
+ columnTypes.put("z", TypeInfoFactory.getDecimalTypeInfo(4, 2));
+ assertEquals(null,
+ ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema, columnTypes));
+
+ sarg = SearchArgumentFactory.newBuilder()
+ .startNot()
+ .startOr()
+ .isNull("x", PredicateLeaf.Type.LONG)
+ .between("y", PredicateLeaf.Type.DECIMAL, new HiveDecimalWritable("10"),
+ new HiveDecimalWritable("20.0"))
+ .in("z", PredicateLeaf.Type.LONG, 1L, 2L, 3L)
+ .nullSafeEquals("a", PredicateLeaf.Type.STRING,
+ new HiveVarchar("stinger", 100).toString())
+ .end()
+ .end()
+ .build();
+ schema = MessageTypeParser.parseMessageType("message test {" +
+ " optional int32 x; required binary y; required int32 z;" +
+ " optional binary a;}");
+ columnTypes = new HashMap<>();
+ columnTypes.put("x", TypeInfoFactory.getPrimitiveTypeInfo("int"));
+ columnTypes.put("y", TypeInfoFactory.getDecimalTypeInfo(4, 2));
+ columnTypes.put("z", TypeInfoFactory.getPrimitiveTypeInfo("int"));
+ columnTypes.put("a", TypeInfoFactory.getCharTypeInfo(100));
+ assertEquals(null,
+ ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema, columnTypes));
+ }
+
+ @Test
+ public void testFilterFloatColumn() throws Exception {
+ SearchArgument sarg =
+ SearchArgumentFactory.newBuilder()
+ .startAnd()
+ .lessThan("x", PredicateLeaf.Type.LONG, 22L)
+ .lessThan("x1", PredicateLeaf.Type.LONG, 22L)
+ .lessThanEquals("y", PredicateLeaf.Type.STRING,
+ new HiveChar("hi", 10).toString())
+ .equals("z", PredicateLeaf.Type.FLOAT, Double.valueOf(0.22))
+ .equals("z1", PredicateLeaf.Type.FLOAT, Double.valueOf(0.22))
+ .end()
+ .build();
+ MessageType schema = MessageTypeParser.parseMessageType("message test {" +
+ " required int32 x; required int32 x1;" +
+ " required binary y; required float z; required float z1;}");
+ Map<String, TypeInfo> columnTypes = new HashMap<>();
+ columnTypes.put("x", TypeInfoFactory.getPrimitiveTypeInfo("int"));
+ columnTypes.put("x1", TypeInfoFactory.getPrimitiveTypeInfo("int"));
+ columnTypes.put("y", TypeInfoFactory.getCharTypeInfo(10));
+ columnTypes.put("z", TypeInfoFactory.getPrimitiveTypeInfo("float"));
+ columnTypes.put("z1", TypeInfoFactory.getPrimitiveTypeInfo("float"));
+
+ FilterPredicate p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema, columnTypes);
+ 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());
+ }
+
+ @Test
+ public void testFilterCharColumnLessThan() throws Exception {
+ SearchArgument sarg =
+ SearchArgumentFactory.newBuilder()
+ .lessThan("a", PredicateLeaf.Type.STRING, new HiveChar("apple", 10).toString())
+ .build();
+ MessageType schema = MessageTypeParser.parseMessageType(
+ "message test {required binary a;}");
+
+ Map<String, TypeInfo> columnTypes = new HashMap<>();
+ columnTypes.put("a", TypeInfoFactory.getCharTypeInfo(10));
+
+ FilterPredicate p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema, columnTypes);
+ String expected = "lt(a, Binary{\"apple\"})";
+ assertEquals(expected, p.toString());
+ }
+
+ @Test
+ public void testFilterCharColumnLessThanEquals() throws Exception {
+ SearchArgument sarg =
+ SearchArgumentFactory.newBuilder()
+ .lessThanEquals("a", PredicateLeaf.Type.STRING, new HiveChar("apple", 10).toString())
+ .build();
+ MessageType schema = MessageTypeParser.parseMessageType(
+ "message test {required binary a;}");
+
+ Map<String, TypeInfo> columnTypes = new HashMap<>();
+ columnTypes.put("a", TypeInfoFactory.getCharTypeInfo(10));
+
+ FilterPredicate p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema, columnTypes);
+ String expected = "lteq(a, Binary{\"apple\"})";
+ assertEquals(expected, p.toString());
+ }
+
+ @Test
+ public void testFilterCharColumnEquals() throws Exception {
+ SearchArgument sarg =
+ SearchArgumentFactory.newBuilder()
+ .equals("a", PredicateLeaf.Type.STRING, new HiveChar("apple", 10).toString())
+ .build();
+ MessageType schema = MessageTypeParser.parseMessageType(
+ "message test {required binary a;}");
+
+ Map<String, TypeInfo> columnTypes = new HashMap<>();
+ columnTypes.put("a", TypeInfoFactory.getCharTypeInfo(10));
+
+ FilterPredicate p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema, columnTypes);
+ String expected = "eq(a, Binary{\"apple\"})";
+ assertEquals(expected, p.toString());
+ }
+
+ @Test
+ public void testFilterCharColumnNullSafeEquals() throws Exception {
+ SearchArgument sarg =
+ SearchArgumentFactory.newBuilder()
+ .nullSafeEquals("a", PredicateLeaf.Type.STRING, new HiveChar("apple", 10).toString())
+ .build();
+ MessageType schema = MessageTypeParser.parseMessageType(
+ "message test {required binary a;}");
+
+ Map<String, TypeInfo> columnTypes = new HashMap<>();
+ columnTypes.put("a", TypeInfoFactory.getCharTypeInfo(10));
+
+ FilterPredicate p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema, columnTypes);
+ String expected = "eq(a, Binary{\"apple\"})";
+ assertEquals(expected, p.toString());
+ }
+
+ @Test
+ public void testFilterCharColumnIn() throws Exception {
+ SearchArgument sarg =
+ SearchArgumentFactory.newBuilder()
+ .in("a", PredicateLeaf.Type.STRING, new HiveChar("cherry", 10).toString(), new HiveChar("orange", 10).toString())
+ .build();
+ MessageType schema = MessageTypeParser.parseMessageType(
+ "message test {required binary a;}");
+
+ Map<String, TypeInfo> columnTypes = new HashMap<>();
+ columnTypes.put("a", TypeInfoFactory.getCharTypeInfo(10));
+
+ FilterPredicate p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema, columnTypes);
+ String expected = "or(eq(a, Binary{\"cherry\"}), eq(a, Binary{\"orange\"}))";
+ assertEquals(expected, p.toString());
+ }
+
+ @Test
+ public void testFilterCharColumnBetween() throws Exception {
+ SearchArgument sarg =
+ SearchArgumentFactory.newBuilder()
+ .between("a", PredicateLeaf.Type.STRING, new HiveChar("apple", 10).toString(), new HiveChar("pear", 10).toString())
+ .build();
+ MessageType schema = MessageTypeParser.parseMessageType(
+ "message test {required binary a;}");
+
+ Map<String, TypeInfo> columnTypes = new HashMap<>();
+ columnTypes.put("a", TypeInfoFactory.getCharTypeInfo(10));
+
+ FilterPredicate p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema, columnTypes);
+ String expected = "and(lteq(a, Binary{\"pear\"}), not(lt(a, Binary{\"apple\"})))";
+ assertEquals(expected, p.toString());
+ }
+
+ @Test
+ public void testFilterCharColumnIsNull() throws Exception {
+ SearchArgument sarg =
+ SearchArgumentFactory.newBuilder()
+ .isNull("a", PredicateLeaf.Type.STRING)
+ .build();
+ MessageType schema = MessageTypeParser.parseMessageType(
+ "message test {required binary a;}");
+
+ Map<String, TypeInfo> columnTypes = new HashMap<>();
+ columnTypes.put("a", TypeInfoFactory.getCharTypeInfo(10));
+
+ FilterPredicate p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema, columnTypes);
+ String expected = "eq(a, null)";
+ assertEquals(expected, p.toString());
+ }
+
+ @Test
+ public void testFilterCharColumnGreaterThan() throws Exception {
+ SearchArgument sarg =
+ SearchArgumentFactory.newBuilder()
+ .startNot()
+ .lessThanEquals("a", PredicateLeaf.Type.STRING, new HiveChar("apple", 10).toString())
+ .end()
+ .build();
+ MessageType schema = MessageTypeParser.parseMessageType(
+ "message test {required binary a;}");
+
+ Map<String, TypeInfo> columnTypes = new HashMap<>();
+ columnTypes.put("a", TypeInfoFactory.getCharTypeInfo(10));
+
+ FilterPredicate p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema, columnTypes);
+ String expected = "not(lteq(a, Binary{\"apple\"}))";
+ assertEquals(expected, p.toString());
+ }
+
+ @Test
+ public void testFilterCharColumnWhiteSpacePrefix() throws Exception {
+ SearchArgument sarg =
+ SearchArgumentFactory.newBuilder()
+ .startAnd()
+ .lessThan("a", PredicateLeaf.Type.STRING, new HiveChar(" apple", 10).toString())
+ .lessThanEquals("b", PredicateLeaf.Type.STRING, new HiveChar(" pear", 10).toString())
+ .equals("c", PredicateLeaf.Type.STRING, new HiveChar(" orange", 10).toString())
+ .nullSafeEquals("d", PredicateLeaf.Type.STRING, new HiveChar(" pineapple", 10).toString())
+ .end()
+ .build();
+ MessageType schema = MessageTypeParser.parseMessageType("message test {" +
+ " required binary a; required binary b;"
+ + " required binary c; required binary d;}");
+ Map<String, TypeInfo> columnTypes = new HashMap<>();
+ columnTypes.put("a", TypeInfoFactory.getCharTypeInfo(10));
+ columnTypes.put("b", TypeInfoFactory.getCharTypeInfo(10));
+ columnTypes.put("c", TypeInfoFactory.getCharTypeInfo(10));
+ columnTypes.put("d", TypeInfoFactory.getCharTypeInfo(10));
+
+ FilterPredicate p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema, columnTypes);
+ String expected = "and(and(and("
+ + "lt(a, Binary{\" apple\"}), "
+ + "lteq(b, Binary{\" pear\"})), "
+ + "eq(c, Binary{\" orange\"})), "
+ + "eq(d, Binary{\" pineapple\"}))";
+ assertEquals(expected, p.toString());
+ }
+
+ @Test
+ public void testFilterCharColumnWhiteSpacePostfix() throws Exception {
+ SearchArgument sarg =
+ SearchArgumentFactory.newBuilder()
+ .startAnd()
+ .lessThan("a", PredicateLeaf.Type.STRING, new HiveChar("apple ", 10).toString())
+ .lessThanEquals("b", PredicateLeaf.Type.STRING, new HiveChar("pear ", 10).toString())
+ .equals("c", PredicateLeaf.Type.STRING, new HiveChar("orange ", 10).toString())
+ .nullSafeEquals("d", PredicateLeaf.Type.STRING, new HiveChar("pineapple ", 10).toString())
+ .end()
+ .build();
+ MessageType schema = MessageTypeParser.parseMessageType("message test {" +
+ " required binary a; required binary b;"
+ + " required binary c; required binary d;}");
+ Map<String, TypeInfo> columnTypes = new HashMap<>();
+ columnTypes.put("a", TypeInfoFactory.getCharTypeInfo(10));
+ columnTypes.put("b", TypeInfoFactory.getCharTypeInfo(10));
+ columnTypes.put("c", TypeInfoFactory.getCharTypeInfo(10));
+ columnTypes.put("d", TypeInfoFactory.getCharTypeInfo(10));
+
+ FilterPredicate p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema, columnTypes);
+ String expected = "and(and(and("
+ + "lt(a, Binary{\"apple\"}), "
+ + "lteq(b, Binary{\"pear\"})), "
+ + "eq(c, Binary{\"orange\"})), "
+ + "eq(d, Binary{\"pineapple\"}))";
+ assertEquals(expected, p.toString());
+ }
+ @Test
+ public void testFilterMoreComplexCharColumn() throws Exception {
+ //((a=pear or a<=cherry) and (b=orange)) and (c=banana or d<cherry)
+ SearchArgument sarg =
+ SearchArgumentFactory.newBuilder()
+ .startAnd()
+ .startAnd()
+ .startOr()
+ .equals("a", PredicateLeaf.Type.STRING, new HiveChar("pear", 10).toString())
+ .lessThanEquals("a", PredicateLeaf.Type.STRING, new HiveChar("cherry", 10).toString())
+ .end()
+ .equals("b", PredicateLeaf.Type.STRING, new HiveChar("orange", 10).toString())
+ .end()
+ .startOr()
+ .equals("c", PredicateLeaf.Type.STRING, new HiveChar("banana", 10).toString())
+ .lessThan("d", PredicateLeaf.Type.STRING, new HiveChar("cherry", 10).toString())
+ .end()
+ .end()
+ .build();
+ MessageType schema = MessageTypeParser.parseMessageType("message test {" +
+ " required binary a; required binary b;"
+ + " required binary c; required binary d;}");
+ Map<String, TypeInfo> columnTypes = new HashMap<>();
+ columnTypes.put("a", TypeInfoFactory.getCharTypeInfo(10));
+ columnTypes.put("b", TypeInfoFactory.getCharTypeInfo(10));
+ columnTypes.put("c", TypeInfoFactory.getCharTypeInfo(10));
+ columnTypes.put("d", TypeInfoFactory.getCharTypeInfo(10));
+
+ FilterPredicate p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema, columnTypes);
+ String expected = "and(and(or("
+ + "eq(a, Binary{\"pear\"}), "
+ + "lteq(a, Binary{\"cherry\"})), "
+ + "eq(b, Binary{\"orange\"})), "
+ + "or(eq(c, Binary{\"banana\"}), lt(d, Binary{\"cherry\"})))";
+ assertEquals(expected, p.toString());
+ }
+
+ @Test
+ public void testFilterVarCharColumn() throws Exception {
+ SearchArgument sarg =
+ SearchArgumentFactory.newBuilder()
+ .startAnd()
+ .lessThan("a", PredicateLeaf.Type.STRING, new HiveVarchar("apple", 10).toString())
+ .lessThanEquals("b", PredicateLeaf.Type.STRING, new HiveVarchar("pear", 10).toString())
+ .equals("c", PredicateLeaf.Type.STRING, new HiveVarchar("orange", 10).toString())
+ .nullSafeEquals("d", PredicateLeaf.Type.STRING, new HiveVarchar("pineapple", 9).toString())
+ .in("e", PredicateLeaf.Type.STRING, new HiveVarchar("cherry", 10).toString(), new HiveVarchar("orange", 10).toString())
+ .between("f", PredicateLeaf.Type.STRING, new HiveVarchar("apple", 10).toString(), new HiveVarchar("pear", 10).toString())
+ .isNull("g", PredicateLeaf.Type.STRING)
+ .end()
+ .build();
+ MessageType schema = MessageTypeParser.parseMessageType("message test {" +
+ " required binary a; required binary b;"
+ + " required binary c; required binary d;"
+ + " required binary e; required binary f;"
+ + " required binary g;}");
+
+ Map<String, TypeInfo> columnTypes = new HashMap<>();
+ columnTypes.put("a", TypeInfoFactory.getVarcharTypeInfo(10));
+ columnTypes.put("b", TypeInfoFactory.getVarcharTypeInfo(10));
+ columnTypes.put("c", TypeInfoFactory.getVarcharTypeInfo(10));
+ columnTypes.put("d", TypeInfoFactory.getVarcharTypeInfo(10));
+ columnTypes.put("e", TypeInfoFactory.getVarcharTypeInfo(10));
+ columnTypes.put("f", TypeInfoFactory.getVarcharTypeInfo(10));
+ columnTypes.put("g", TypeInfoFactory.getVarcharTypeInfo(10));
+
+ FilterPredicate p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema, columnTypes);
+ String expected = "and(and(and(and(and(and("
+ + "lt(a, Binary{\"apple\"}), "
+ + "lteq(b, Binary{\"pear\"})), "
+ + "eq(c, Binary{\"orange\"})), "
+ + "eq(d, Binary{\"pineapple\"})), "
+ + "or(eq(e, Binary{\"cherry\"}), eq(e, Binary{\"orange\"}))), "
+ + "and(lteq(f, Binary{\"pear\"}), not(lt(f, Binary{\"apple\"})))), "
+ + "eq(g, null))";
+ assertEquals(expected, p.toString());
+ }
+
+ @Test
+ public void testFilterVarCharColumnWithWhiteSpaces() throws Exception {
+ SearchArgument sarg =
+ SearchArgumentFactory.newBuilder()
+ .startAnd()
+ .lessThan("a", PredicateLeaf.Type.STRING, new HiveVarchar(" apple ", 10).toString())
+ .lessThanEquals("b", PredicateLeaf.Type.STRING, new HiveVarchar(" pear", 10).toString())
+ .equals("c", PredicateLeaf.Type.STRING, new HiveVarchar("orange ", 10).toString())
+ .end()
+ .build();
+ MessageType schema = MessageTypeParser.parseMessageType("message test {" +
+ " required binary a; required binary b;"
+ + " required binary c;}");
+ Map<String, TypeInfo> columnTypes = new HashMap<>();
+ columnTypes.put("a", TypeInfoFactory.getVarcharTypeInfo(10));
+ columnTypes.put("b", TypeInfoFactory.getVarcharTypeInfo(10));
+ columnTypes.put("c", TypeInfoFactory.getVarcharTypeInfo(10));
+
+ FilterPredicate p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema, columnTypes);
+ String expected = "and(and("
+ + "lt(a, Binary{\" apple \"}), "
+ + "lteq(b, Binary{\" pear\"})), "
+ + "eq(c, Binary{\"orange \"}))";
+ assertEquals(expected, p.toString());
+ }
+
+ @Test
+ public void testFilterStringColumnWithWhiteSpaces() throws Exception {
+ SearchArgument sarg =
+ SearchArgumentFactory.newBuilder()
+ .startAnd()
+ .lessThan("a", PredicateLeaf.Type.STRING, new HiveVarchar(" apple ", 10).toString())
+ .lessThanEquals("b", PredicateLeaf.Type.STRING, new HiveVarchar(" pear", 10).toString())
+ .equals("c", PredicateLeaf.Type.STRING, new HiveVarchar("orange ", 10).toString())
+ .end()
+ .build();
+ MessageType schema = MessageTypeParser.parseMessageType("message test {" +
+ " required binary a; required binary b;"
+ + " required binary c;}");
+ Map<String, TypeInfo> columnTypes = new HashMap<>();
+ columnTypes.put("a", TypeInfoFactory.getPrimitiveTypeInfo("string"));
+ columnTypes.put("b", TypeInfoFactory.getPrimitiveTypeInfo("string"));
+ columnTypes.put("c", TypeInfoFactory.getPrimitiveTypeInfo("string"));
+
+ FilterPredicate p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema, columnTypes);
+ String expected = "and(and("
+ + "lt(a, Binary{\" apple \"}), "
+ + "lteq(b, Binary{\" pear\"})), "
+ + "eq(c, Binary{\"orange \"}))";
+ assertEquals(expected, p.toString());
+ }
+
+ @Test
+ public void testFilterMoreComplexVarCharColumn() throws Exception {
+ //((a=pear or a<=cherry) and (b=orange)) and (c=banana or d<cherry)
+ SearchArgument sarg =
+ SearchArgumentFactory.newBuilder()
+ .startAnd()
+ .startAnd()
+ .startOr()
+ .equals("a", PredicateLeaf.Type.STRING, new HiveVarchar("pear", 10).toString())
+ .lessThanEquals("a", PredicateLeaf.Type.STRING, new HiveVarchar("cherry", 10).toString())
+ .end()
+ .equals("b", PredicateLeaf.Type.STRING, new HiveVarchar("orange", 10).toString())
+ .end()
+ .startOr()
+ .equals("c", PredicateLeaf.Type.STRING, new HiveVarchar("banana", 10).toString())
+ .lessThan("d", PredicateLeaf.Type.STRING, new HiveVarchar("cherry", 10).toString())
+ .end()
+ .end()
+ .build();
+ MessageType schema = MessageTypeParser.parseMessageType("message test {" +
+ " required binary a; required binary b;"
+ + " required binary c; required binary d;}");
+
+ Map<String, TypeInfo> columnTypes = new HashMap<>();
+ columnTypes.put("a", TypeInfoFactory.getVarcharTypeInfo(10));
+ columnTypes.put("b", TypeInfoFactory.getVarcharTypeInfo(10));
+ columnTypes.put("c", TypeInfoFactory.getVarcharTypeInfo(10));
+ columnTypes.put("d", TypeInfoFactory.getVarcharTypeInfo(10));
+
+ FilterPredicate p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema, columnTypes);
+ String expected = "and(and(or(eq(a, Binary{\"pear\"}), lteq(a, Binary{\"cherry\"})), "
+ + "eq(b, Binary{\"orange\"})), "
+ + "or(eq(c, Binary{\"banana\"}), lt(d, Binary{\"cherry\"})))";
+ assertEquals(expected, p.toString());
+ }
}
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/sarg/TestConvertAstToSearchArg.java b/ql/src/test/org/apache/hadoop/hive/ql/io/sarg/TestConvertAstToSearchArg.java
index 777382c76c3..659af68ccc7 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/sarg/TestConvertAstToSearchArg.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/sarg/TestConvertAstToSearchArg.java
@@ -18,14 +18,16 @@
package org.apache.hadoop.hive.ql.io.sarg;
-import static junit.framework.Assert.assertEquals;
-import static junit.framework.Assert.assertNull;
-import static junit.framework.Assert.assertTrue;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
import java.beans.XMLDecoder;
import java.io.ByteArrayInputStream;
import java.io.UnsupportedEncodingException;
+import java.util.HashMap;
import java.util.List;
+import java.util.Map;
import java.util.Set;
import org.apache.hadoop.conf.Configuration;
@@ -33,6 +35,8 @@ import org.apache.hadoop.hive.ql.exec.SerializationUtilities;
import org.apache.hadoop.hive.ql.io.parquet.read.ParquetFilterPredicateConverter;
import org.apache.hadoop.hive.ql.io.sarg.SearchArgument.TruthValue;
import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
import org.apache.parquet.filter2.predicate.FilterPredicate;
import org.apache.parquet.schema.MessageType;
import org.apache.parquet.schema.MessageTypeParser;
@@ -558,7 +562,11 @@ public class TestConvertAstToSearchArg {
MessageTypeParser.parseMessageType("message test { required int32 id;" +
" required binary first_name; }");
- FilterPredicate p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema);
+ Map<String, TypeInfo> columnTypes = new HashMap<>();
+ columnTypes.put("id", TypeInfoFactory.getPrimitiveTypeInfo("int"));
+ columnTypes.put("first_name", TypeInfoFactory.getPrimitiveTypeInfo("string"));
+
+ FilterPredicate p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema, columnTypes);
String[] conditions = new String[]{
"eq(first_name, Binary{\"john\"})", /* first_name = 'john' */
"not(lteq(first_name, Binary{\"greg\"}))", /* 'greg' < first_name */
@@ -853,7 +861,11 @@ public class TestConvertAstToSearchArg {
MessageType schema =
MessageTypeParser.parseMessageType("message test { required int32 id;" +
" required binary first_name; }");
- FilterPredicate p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema);
+ Map<String, TypeInfo> columnTypes = new HashMap<>();
+ columnTypes.put("id", TypeInfoFactory.getPrimitiveTypeInfo("int"));
+ columnTypes.put("first_name", TypeInfoFactory.getPrimitiveTypeInfo("string"));
+
+ FilterPredicate p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema, columnTypes);
String expected = String.format("or(or(or(%1$s, %2$s), %3$s), %4$s)", conditions);
assertEquals(expected, p.toString());
@@ -1285,8 +1297,12 @@ public class TestConvertAstToSearchArg {
MessageType schema =
MessageTypeParser.parseMessageType("message test { required int32 id;" +
" required binary first_name; required binary last_name;}");
+ Map<String, TypeInfo> columnTypes = new HashMap<>();
+ columnTypes.put("id", TypeInfoFactory.getPrimitiveTypeInfo("int"));
+ columnTypes.put("first_name", TypeInfoFactory.getPrimitiveTypeInfo("string"));
+ columnTypes.put("last_name", TypeInfoFactory.getPrimitiveTypeInfo("string"));
- FilterPredicate p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema);
+ FilterPredicate p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema, columnTypes);
String expected = String.format("and(and(and(%1$s, %2$s), %3$s), %4$s)", conditions);
assertEquals(expected, p.toString());
@@ -1510,8 +1526,12 @@ public class TestConvertAstToSearchArg {
MessageType schema =
MessageTypeParser.parseMessageType("message test { required int32 id;" +
" required binary first_name; }");
+ Map<String, TypeInfo> columnTypes = new HashMap<>();
+ columnTypes.put("id", TypeInfoFactory.getPrimitiveTypeInfo("int"));
+ columnTypes.put("first_name", TypeInfoFactory.getPrimitiveTypeInfo("string"));
+
FilterPredicate p =
- ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema);
+ ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema, columnTypes);
String expected = String.format("and(and(%1$s, %2$s), %3$s)", conditions);
assertEquals(expected, p.toString());
@@ -1773,7 +1793,12 @@ public class TestConvertAstToSearchArg {
MessageType schema =
MessageTypeParser.parseMessageType("message test { required int32 id;" +
" required binary first_name; }");
- FilterPredicate p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema);
+ Map<String, TypeInfo> columnTypes = new HashMap<>();
+ columnTypes.put("id", TypeInfoFactory.getPrimitiveTypeInfo("int"));
+ columnTypes.put("first_name", TypeInfoFactory.getPrimitiveTypeInfo("string"));
+
+ FilterPredicate p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema, columnTypes
+ );
String expected =
"and(lteq(first_name, Binary{\"greg\"}), not(lt(first_name, Binary{\"david\"})))";
assertEquals(p.toString(), expected);
@@ -2256,7 +2281,11 @@ public class TestConvertAstToSearchArg {
MessageType schema =
MessageTypeParser.parseMessageType("message test { required int32 id;" +
" required binary first_name; }");
- FilterPredicate p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema);
+ Map<String, TypeInfo> columnTypes = new HashMap<>();
+ columnTypes.put("id", TypeInfoFactory.getPrimitiveTypeInfo("int"));
+ columnTypes.put("first_name", TypeInfoFactory.getPrimitiveTypeInfo("string"));
+
+ FilterPredicate p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema, columnTypes);
String expected = "and(and(and(and(and(and(and(and(and(and(and(and(and(and(and(and(and(" +
"or(or(or(lt(id, 18), lt(id, 10)), lt(id, 13)), lt(id, 16)), " +
"or(or(or(lt(id, 18), lt(id, 11)), lt(id, 13)), lt(id, 16))), " +
@@ -2415,7 +2444,11 @@ public class TestConvertAstToSearchArg {
MessageType schema =
MessageTypeParser.parseMessageType("message test { required int32 id;" +
" required binary first_name; }");
- FilterPredicate p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema);
+ Map<String, TypeInfo> columnTypes = new HashMap<>();
+ columnTypes.put("id", TypeInfoFactory.getPrimitiveTypeInfo("int"));
+ columnTypes.put("first_name", TypeInfoFactory.getPrimitiveTypeInfo("string"));
+
+ FilterPredicate p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema, columnTypes);
assertNull(p);
assertEquals("YES_NO_NULL",
@@ -2673,7 +2706,11 @@ public class TestConvertAstToSearchArg {
MessageType schema =
MessageTypeParser.parseMessageType("message test { required int32 id;" +
" required binary first_name; }");
- FilterPredicate p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema);
+ Map<String, TypeInfo> columnTypes = new HashMap<>();
+ columnTypes.put("id", TypeInfoFactory.getPrimitiveTypeInfo("int"));
+ columnTypes.put("first_name", TypeInfoFactory.getPrimitiveTypeInfo("string"));
+
+ FilterPredicate p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema, columnTypes);
String expected = "and(not(lt(id, 10)), not(lt(id, 10)))";
assertEquals(expected, p.toString());
diff --git a/ql/src/test/queries/clientpositive/parquet_ppd_char2.q b/ql/src/test/queries/clientpositive/parquet_ppd_char2.q
new file mode 100644
index 00000000000..ef965d75ad5
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/parquet_ppd_char2.q
@@ -0,0 +1,119 @@
+SET hive.optimize.index.filter=true;
+SET hive.optimize.ppd=true;
+SET hive.optimize.ppd.storage=true;
+SET hive.vectorized.execution.enabled=false;
+SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
+SET mapred.min.split.size=1000;
+SET mapred.max.split.size=5000;
+
+drop table if exists ppd_char_test;
+create table ppd_char_test (id int, a char(10), b char(10), c varchar(10), d varchar(10)) stored as parquet;
+
+insert into ppd_char_test values(1, 'apple', 'orange', 'lemon', 'apple'),
+(2, 'almond', 'cherry', 'banana', 'apple'),
+(3, 'banana', 'orange', 'banana ', 'pear'),
+(4, 'cherry', 'orange', 'banana', 'lemon'),
+(5, 'lemon', 'orange', 'banana', 'apple'),
+(6, 'orange', 'orange', 'banana', 'apple'),
+(7, 'pear', 'orange', 'banana', 'apple'),
+(8, 'pear', 'orange', 'lemon', 'apple '),
+(9, 'pear', 'orange', 'banana', 'pear'),
+(10, 'pear', 'cherry', 'banana', 'apple'),
+(11, 'pineapple', 'cherry', 'lemon', 'apple'),
+(12, 'pineapple', 'cherry', ' lemon ', 'apple'),
+(13, 'pineapple', 'cherry', 'lemon ', 'apple'),
+(14, 'pineapple', 'cherry', ' lemon ', 'apple');
+
+select * from ppd_char_test where ((a='pear' or a<='cherry') and (b='orange')) and (c='banana' or d<'cherry') order by id;
+
+select * from ppd_char_test where ((a='pear' or a<='cherry') and (b='orange')) and c='banana ' order by id;
+
+select id, a from ppd_char_test where a='apple';
+
+select id, a from ppd_char_test where a!='apple';
+
+select id, a from ppd_char_test where a<'cherry';
+
+select id, a from ppd_char_test where a<='cherry';
+
+select id, a from ppd_char_test where a>'cherry';
+
+select id, a from ppd_char_test where a>='cherry';
+
+select id, c from ppd_char_test where a='pineapple' and c='lemon';
+
+select id, c from ppd_char_test where a='pineapple' and c!='lemon';
+
+select id, c from ppd_char_test where a='pineapple' and c!=' lemon ';
+
+select id, c from ppd_char_test where a='pineapple' and c=' lemon';
+
+select id, c from ppd_char_test where a='pineapple' and c='lemon ';
+
+select id, c from ppd_char_test where a='pineapple' and c='lemon ';
+
+select id, c from ppd_char_test where a='pineapple' and c='lemon ';
+
+select id, c from ppd_char_test where a='pineapple' and c=' lemon ';
+
+select id, c from ppd_char_test where a='pineapple' and c<'lemon';
+
+select id, c from ppd_char_test where a='pineapple' and c<='lemon';
+
+select id, c from ppd_char_test where a='pineapple' and c>'lemon';
+
+select id, c from ppd_char_test where a='pineapple' and c>='lemon';
+
+select id, c from ppd_char_test where a='pineapple' and c<='lemon ';
+
+select id, c from ppd_char_test where a='pineapple' and c>'lemon ';
+
+SET hive.optimize.index.filter=false;
+SET hive.optimize.ppd=false;
+SET hive.optimize.ppd.storage=false;
+
+select * from ppd_char_test where ((a='pear' or a<='cherry') and (b='orange')) and (c='banana' or d<'cherry') order by id;
+
+select * from ppd_char_test where ((a='pear' or a<='cherry') and (b='orange')) and c='banana ' order by id;
+
+select id, a from ppd_char_test where a='apple';
+
+select id, a from ppd_char_test where a!='apple';
+
+select id, a from ppd_char_test where a<'cherry';
+
+select id, a from ppd_char_test where a<='cherry';
+
+select id, a from ppd_char_test where a>'cherry';
+
+select id, a from ppd_char_test where a>='cherry';
+
+select id, c from ppd_char_test where a='pineapple' and c='lemon';
+
+select id, c from ppd_char_test where a='pineapple' and c!='lemon';
+
+select id, c from ppd_char_test where a='pineapple' and c!=' lemon ';
+
+select id, c from ppd_char_test where a='pineapple' and c=' lemon';
+
+select id, c from ppd_char_test where a='pineapple' and c='lemon ';
+
+select id, c from ppd_char_test where a='pineapple' and c='lemon ';
+
+select id, c from ppd_char_test where a='pineapple' and c='lemon ';
+
+select id, c from ppd_char_test where a='pineapple' and c=' lemon ';
+
+select id, c from ppd_char_test where a='pineapple' and c<'lemon';
+
+select id, c from ppd_char_test where a='pineapple' and c<='lemon';
+
+select id, c from ppd_char_test where a='pineapple' and c>'lemon';
+
+select id, c from ppd_char_test where a='pineapple' and c>='lemon';
+
+select id, c from ppd_char_test where a='pineapple' and c<='lemon ';
+
+select id, c from ppd_char_test where a='pineapple' and c>'lemon ';
+
+drop table ppd_char_test;
diff --git a/ql/src/test/results/clientpositive/parquet_ppd_char2.q.out b/ql/src/test/results/clientpositive/parquet_ppd_char2.q.out
new file mode 100644
index 00000000000..7d563cf3469
--- /dev/null
+++ b/ql/src/test/results/clientpositive/parquet_ppd_char2.q.out
@@ -0,0 +1,551 @@
+PREHOOK: query: drop table if exists ppd_char_test
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table if exists ppd_char_test
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: create table ppd_char_test (id int, a char(10), b char(10), c varchar(10), d varchar(10)) stored as parquet
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@ppd_char_test
+POSTHOOK: query: create table ppd_char_test (id int, a char(10), b char(10), c varchar(10), d varchar(10)) stored as parquet
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@ppd_char_test
+PREHOOK: query: insert into ppd_char_test values(1, 'apple', 'orange', 'lemon', 'apple'),
+(2, 'almond', 'cherry', 'banana', 'apple'),
+(3, 'banana', 'orange', 'banana ', 'pear'),
+(4, 'cherry', 'orange', 'banana', 'lemon'),
+(5, 'lemon', 'orange', 'banana', 'apple'),
+(6, 'orange', 'orange', 'banana', 'apple'),
+(7, 'pear', 'orange', 'banana', 'apple'),
+(8, 'pear', 'orange', 'lemon', 'apple '),
+(9, 'pear', 'orange', 'banana', 'pear'),
+(10, 'pear', 'cherry', 'banana', 'apple'),
+(11, 'pineapple', 'cherry', 'lemon', 'apple'),
+(12, 'pineapple', 'cherry', ' lemon ', 'apple'),
+(13, 'pineapple', 'cherry', 'lemon ', 'apple'),
+(14, 'pineapple', 'cherry', ' lemon ', 'apple')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ppd_char_test
+POSTHOOK: query: insert into ppd_char_test values(1, 'apple', 'orange', 'lemon', 'apple'),
+(2, 'almond', 'cherry', 'banana', 'apple'),
+(3, 'banana', 'orange', 'banana ', 'pear'),
+(4, 'cherry', 'orange', 'banana', 'lemon'),
+(5, 'lemon', 'orange', 'banana', 'apple'),
+(6, 'orange', 'orange', 'banana', 'apple'),
+(7, 'pear', 'orange', 'banana', 'apple'),
+(8, 'pear', 'orange', 'lemon', 'apple '),
+(9, 'pear', 'orange', 'banana', 'pear'),
+(10, 'pear', 'cherry', 'banana', 'apple'),
+(11, 'pineapple', 'cherry', 'lemon', 'apple'),
+(12, 'pineapple', 'cherry', ' lemon ', 'apple'),
+(13, 'pineapple', 'cherry', 'lemon ', 'apple'),
+(14, 'pineapple', 'cherry', ' lemon ', 'apple')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ppd_char_test
+POSTHOOK: Lineage: ppd_char_test.a SCRIPT []
+POSTHOOK: Lineage: ppd_char_test.b SCRIPT []
+POSTHOOK: Lineage: ppd_char_test.c SCRIPT []
+POSTHOOK: Lineage: ppd_char_test.d SCRIPT []
+POSTHOOK: Lineage: ppd_char_test.id SCRIPT []
+PREHOOK: query: select * from ppd_char_test where ((a='pear' or a<='cherry') and (b='orange')) and (c='banana' or d<'cherry') order by id
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+POSTHOOK: query: select * from ppd_char_test where ((a='pear' or a<='cherry') and (b='orange')) and (c='banana' or d<'cherry') order by id
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+1 apple orange lemon apple
+4 cherry orange banana lemon
+7 pear orange banana apple
+8 pear orange lemon apple
+9 pear orange banana pear
+PREHOOK: query: select * from ppd_char_test where ((a='pear' or a<='cherry') and (b='orange')) and c='banana ' order by id
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+POSTHOOK: query: select * from ppd_char_test where ((a='pear' or a<='cherry') and (b='orange')) and c='banana ' order by id
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+3 banana orange banana pear
+PREHOOK: query: select id, a from ppd_char_test where a='apple'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+POSTHOOK: query: select id, a from ppd_char_test where a='apple'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+1 apple
+PREHOOK: query: select id, a from ppd_char_test where a!='apple'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+POSTHOOK: query: select id, a from ppd_char_test where a!='apple'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+2 almond
+3 banana
+4 cherry
+5 lemon
+6 orange
+7 pear
+8 pear
+9 pear
+10 pear
+11 pineapple
+12 pineapple
+13 pineapple
+14 pineapple
+PREHOOK: query: select id, a from ppd_char_test where a<'cherry'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+POSTHOOK: query: select id, a from ppd_char_test where a<'cherry'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+1 apple
+2 almond
+3 banana
+PREHOOK: query: select id, a from ppd_char_test where a<='cherry'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+POSTHOOK: query: select id, a from ppd_char_test where a<='cherry'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+1 apple
+2 almond
+3 banana
+4 cherry
+PREHOOK: query: select id, a from ppd_char_test where a>'cherry'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+POSTHOOK: query: select id, a from ppd_char_test where a>'cherry'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+5 lemon
+6 orange
+7 pear
+8 pear
+9 pear
+10 pear
+11 pineapple
+12 pineapple
+13 pineapple
+14 pineapple
+PREHOOK: query: select id, a from ppd_char_test where a>='cherry'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+POSTHOOK: query: select id, a from ppd_char_test where a>='cherry'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+4 cherry
+5 lemon
+6 orange
+7 pear
+8 pear
+9 pear
+10 pear
+11 pineapple
+12 pineapple
+13 pineapple
+14 pineapple
+PREHOOK: query: select id, c from ppd_char_test where a='pineapple' and c='lemon'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+POSTHOOK: query: select id, c from ppd_char_test where a='pineapple' and c='lemon'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+11 lemon
+PREHOOK: query: select id, c from ppd_char_test where a='pineapple' and c!='lemon'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+POSTHOOK: query: select id, c from ppd_char_test where a='pineapple' and c!='lemon'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+12 lemon
+13 lemon
+14 lemon
+PREHOOK: query: select id, c from ppd_char_test where a='pineapple' and c!=' lemon '
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+POSTHOOK: query: select id, c from ppd_char_test where a='pineapple' and c!=' lemon '
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+11 lemon
+13 lemon
+14 lemon
+PREHOOK: query: select id, c from ppd_char_test where a='pineapple' and c=' lemon'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+POSTHOOK: query: select id, c from ppd_char_test where a='pineapple' and c=' lemon'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+PREHOOK: query: select id, c from ppd_char_test where a='pineapple' and c='lemon '
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+POSTHOOK: query: select id, c from ppd_char_test where a='pineapple' and c='lemon '
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+PREHOOK: query: select id, c from ppd_char_test where a='pineapple' and c='lemon '
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+POSTHOOK: query: select id, c from ppd_char_test where a='pineapple' and c='lemon '
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+PREHOOK: query: select id, c from ppd_char_test where a='pineapple' and c='lemon '
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+POSTHOOK: query: select id, c from ppd_char_test where a='pineapple' and c='lemon '
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+13 lemon
+PREHOOK: query: select id, c from ppd_char_test where a='pineapple' and c=' lemon '
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+POSTHOOK: query: select id, c from ppd_char_test where a='pineapple' and c=' lemon '
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+14 lemon
+PREHOOK: query: select id, c from ppd_char_test where a='pineapple' and c<'lemon'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+POSTHOOK: query: select id, c from ppd_char_test where a='pineapple' and c<'lemon'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+12 lemon
+14 lemon
+PREHOOK: query: select id, c from ppd_char_test where a='pineapple' and c<='lemon'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+POSTHOOK: query: select id, c from ppd_char_test where a='pineapple' and c<='lemon'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+11 lemon
+12 lemon
+14 lemon
+PREHOOK: query: select id, c from ppd_char_test where a='pineapple' and c>'lemon'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+POSTHOOK: query: select id, c from ppd_char_test where a='pineapple' and c>'lemon'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+13 lemon
+PREHOOK: query: select id, c from ppd_char_test where a='pineapple' and c>='lemon'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+POSTHOOK: query: select id, c from ppd_char_test where a='pineapple' and c>='lemon'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+11 lemon
+13 lemon
+PREHOOK: query: select id, c from ppd_char_test where a='pineapple' and c<='lemon '
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+POSTHOOK: query: select id, c from ppd_char_test where a='pineapple' and c<='lemon '
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+11 lemon
+12 lemon
+13 lemon
+14 lemon
+PREHOOK: query: select id, c from ppd_char_test where a='pineapple' and c>'lemon '
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+POSTHOOK: query: select id, c from ppd_char_test where a='pineapple' and c>'lemon '
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+13 lemon
+PREHOOK: query: select * from ppd_char_test where ((a='pear' or a<='cherry') and (b='orange')) and (c='banana' or d<'cherry') order by id
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+POSTHOOK: query: select * from ppd_char_test where ((a='pear' or a<='cherry') and (b='orange')) and (c='banana' or d<'cherry') order by id
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+1 apple orange lemon apple
+4 cherry orange banana lemon
+7 pear orange banana apple
+8 pear orange lemon apple
+9 pear orange banana pear
+PREHOOK: query: select * from ppd_char_test where ((a='pear' or a<='cherry') and (b='orange')) and c='banana ' order by id
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+POSTHOOK: query: select * from ppd_char_test where ((a='pear' or a<='cherry') and (b='orange')) and c='banana ' order by id
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+3 banana orange banana pear
+PREHOOK: query: select id, a from ppd_char_test where a='apple'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+POSTHOOK: query: select id, a from ppd_char_test where a='apple'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+1 apple
+PREHOOK: query: select id, a from ppd_char_test where a!='apple'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+POSTHOOK: query: select id, a from ppd_char_test where a!='apple'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+2 almond
+3 banana
+4 cherry
+5 lemon
+6 orange
+7 pear
+8 pear
+9 pear
+10 pear
+11 pineapple
+12 pineapple
+13 pineapple
+14 pineapple
+PREHOOK: query: select id, a from ppd_char_test where a<'cherry'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+POSTHOOK: query: select id, a from ppd_char_test where a<'cherry'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+1 apple
+2 almond
+3 banana
+PREHOOK: query: select id, a from ppd_char_test where a<='cherry'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+POSTHOOK: query: select id, a from ppd_char_test where a<='cherry'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+1 apple
+2 almond
+3 banana
+4 cherry
+PREHOOK: query: select id, a from ppd_char_test where a>'cherry'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+POSTHOOK: query: select id, a from ppd_char_test where a>'cherry'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+5 lemon
+6 orange
+7 pear
+8 pear
+9 pear
+10 pear
+11 pineapple
+12 pineapple
+13 pineapple
+14 pineapple
+PREHOOK: query: select id, a from ppd_char_test where a>='cherry'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+POSTHOOK: query: select id, a from ppd_char_test where a>='cherry'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+4 cherry
+5 lemon
+6 orange
+7 pear
+8 pear
+9 pear
+10 pear
+11 pineapple
+12 pineapple
+13 pineapple
+14 pineapple
+PREHOOK: query: select id, c from ppd_char_test where a='pineapple' and c='lemon'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+POSTHOOK: query: select id, c from ppd_char_test where a='pineapple' and c='lemon'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+11 lemon
+PREHOOK: query: select id, c from ppd_char_test where a='pineapple' and c!='lemon'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+POSTHOOK: query: select id, c from ppd_char_test where a='pineapple' and c!='lemon'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+12 lemon
+13 lemon
+14 lemon
+PREHOOK: query: select id, c from ppd_char_test where a='pineapple' and c!=' lemon '
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+POSTHOOK: query: select id, c from ppd_char_test where a='pineapple' and c!=' lemon '
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+11 lemon
+13 lemon
+14 lemon
+PREHOOK: query: select id, c from ppd_char_test where a='pineapple' and c=' lemon'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+POSTHOOK: query: select id, c from ppd_char_test where a='pineapple' and c=' lemon'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+PREHOOK: query: select id, c from ppd_char_test where a='pineapple' and c='lemon '
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+POSTHOOK: query: select id, c from ppd_char_test where a='pineapple' and c='lemon '
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+PREHOOK: query: select id, c from ppd_char_test where a='pineapple' and c='lemon '
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+POSTHOOK: query: select id, c from ppd_char_test where a='pineapple' and c='lemon '
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+PREHOOK: query: select id, c from ppd_char_test where a='pineapple' and c='lemon '
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+POSTHOOK: query: select id, c from ppd_char_test where a='pineapple' and c='lemon '
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+13 lemon
+PREHOOK: query: select id, c from ppd_char_test where a='pineapple' and c=' lemon '
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+POSTHOOK: query: select id, c from ppd_char_test where a='pineapple' and c=' lemon '
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+14 lemon
+PREHOOK: query: select id, c from ppd_char_test where a='pineapple' and c<'lemon'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+POSTHOOK: query: select id, c from ppd_char_test where a='pineapple' and c<'lemon'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+12 lemon
+14 lemon
+PREHOOK: query: select id, c from ppd_char_test where a='pineapple' and c<='lemon'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+POSTHOOK: query: select id, c from ppd_char_test where a='pineapple' and c<='lemon'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+11 lemon
+12 lemon
+14 lemon
+PREHOOK: query: select id, c from ppd_char_test where a='pineapple' and c>'lemon'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+POSTHOOK: query: select id, c from ppd_char_test where a='pineapple' and c>'lemon'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+13 lemon
+PREHOOK: query: select id, c from ppd_char_test where a='pineapple' and c>='lemon'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+POSTHOOK: query: select id, c from ppd_char_test where a='pineapple' and c>='lemon'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+11 lemon
+13 lemon
+PREHOOK: query: select id, c from ppd_char_test where a='pineapple' and c<='lemon '
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+POSTHOOK: query: select id, c from ppd_char_test where a='pineapple' and c<='lemon '
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+11 lemon
+12 lemon
+13 lemon
+14 lemon
+PREHOOK: query: select id, c from ppd_char_test where a='pineapple' and c>'lemon '
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+POSTHOOK: query: select id, c from ppd_char_test where a='pineapple' and c>'lemon '
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ppd_char_test
+#### A masked pattern was here ####
+13 lemon
+PREHOOK: query: drop table ppd_char_test
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@ppd_char_test
+PREHOOK: Output: default@ppd_char_test
+POSTHOOK: query: drop table ppd_char_test
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@ppd_char_test
+POSTHOOK: Output: default@ppd_char_test