You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by xu...@apache.org on 2015/09/09 09:08:53 UTC

[23/50] [abbrv] hive git commit: HIVE-11504: Predicate pushing down doesn't work for float type for Parquet(Ferdinand Xu, reviewed by Sergio Pena and Owen O'Malley)

HIVE-11504: Predicate pushing down doesn't work for float type for Parquet(Ferdinand Xu, reviewed by Sergio Pena and Owen O'Malley)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/8f930e58
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/8f930e58
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/8f930e58

Branch: refs/heads/beeline-cli
Commit: 8f930e588efd6ec937b9ad20fcf09030ae210ec3
Parents: a338f33
Author: Ferdinand Xu <ch...@intel.com>
Authored: Mon Aug 31 21:07:10 2015 -0400
Committer: Ferdinand Xu <ch...@intel.com>
Committed: Mon Aug 31 21:07:10 2015 -0400

----------------------------------------------------------------------
 .../hive/ql/io/parquet/LeafFilterFactory.java   |   29 +-
 .../read/TestParquetFilterPredicate.java        |   21 +
 .../clientpositive/parquet_ppd_partition.q      |    9 +
 .../clientpositive/parquet_predicate_pushdown.q |  297 +++-
 .../clientpositive/parquet_ppd_partition.q.out  |   47 +
 .../parquet_predicate_pushdown.q.out            | 1309 +++++++++++++++++-
 6 files changed, 1660 insertions(+), 52 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/8f930e58/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/LeafFilterFactory.java
----------------------------------------------------------------------
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 1ceea6e..3e00612 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
@@ -31,6 +31,7 @@ import static org.apache.parquet.filter2.predicate.FilterApi.ltEq;
 import static org.apache.parquet.filter2.predicate.FilterApi.binaryColumn;
 import static org.apache.parquet.filter2.predicate.FilterApi.booleanColumn;
 import static org.apache.parquet.filter2.predicate.FilterApi.doubleColumn;
+import static org.apache.parquet.filter2.predicate.FilterApi.floatColumn;
 import static org.apache.parquet.filter2.predicate.FilterApi.intColumn;
 
 public class LeafFilterFactory {
@@ -83,6 +84,25 @@ public class LeafFilterFactory {
     }
   }
 
+  class FloatFilterPredicateLeafBuilder extends FilterPredicateLeafBuilder {
+    @Override
+    public FilterPredicate buildPredict(Operator op, Object constant, String columnName) {
+      switch (op) {
+      case LESS_THAN:
+        return lt(floatColumn(columnName), ((Number) constant).floatValue());
+      case IS_NULL:
+      case EQUALS:
+      case NULL_SAFE_EQUALS:
+        return eq(floatColumn(columnName),
+            (constant == null) ? null : ((Number) constant).floatValue());
+      case LESS_THAN_EQUALS:
+        return ltEq(FilterApi.floatColumn(columnName), ((Number) constant).floatValue());
+      default:
+        throw new RuntimeException("Unknown PredicateLeaf Operator type: " + op);
+      }
+    }
+  }
+
   class DoubleFilterPredicateLeafBuilder extends FilterPredicateLeafBuilder {
 
     @Override
@@ -158,8 +178,13 @@ public class LeafFilterFactory {
         } else {
           return new LongFilterPredicateLeafBuilder();
         }
-      case FLOAT:   // float and double
-        return new DoubleFilterPredicateLeafBuilder();
+      case FLOAT:
+        if (parquetType.asPrimitiveType().getPrimitiveTypeName() ==
+            PrimitiveType.PrimitiveTypeName.FLOAT) {
+          return new FloatFilterPredicateLeafBuilder();
+        } else {
+          return new DoubleFilterPredicateLeafBuilder();
+        }
       case STRING:  // string, char, varchar
         return new BinaryFilterPredicateLeafBuilder();
       case BOOLEAN:

http://git-wip-us.apache.org/repos/asf/hive/blob/8f930e58/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/read/TestParquetFilterPredicate.java
----------------------------------------------------------------------
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 ac5c1a0..2be2596 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
@@ -48,4 +48,25 @@ public class TestParquetFilterPredicate {
     String expected = "and(not(eq(a, null)), not(eq(a, Binary{\"stinger\"})))";
     assertEquals(expected, p.toString());
   }
+
+  @Test
+  public void testFilterFloatColumns() {
+    MessageType schema =
+        MessageTypeParser.parseMessageType("message test {  required float a; required int32 b; }");
+    SearchArgument sarg = SearchArgumentFactory.newBuilder()
+        .startNot()
+        .startOr()
+        .isNull("a", PredicateLeaf.Type.FLOAT)
+        .between("a", PredicateLeaf.Type.FLOAT, 10.2, 20.3)
+        .in("b", PredicateLeaf.Type.LONG, 1L, 2L, 3L)
+        .end()
+        .end()
+        .build();
+
+    FilterPredicate p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema);
+
+    String expected =
+        "and(and(not(eq(a, null)), not(and(lt(a, 20.3), not(lteq(a, 10.2))))), not(or(or(eq(b, 1), eq(b, 2)), eq(b, 3))))";
+    assertEquals(expected, p.toString());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/8f930e58/ql/src/test/queries/clientpositive/parquet_ppd_partition.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/parquet_ppd_partition.q b/ql/src/test/queries/clientpositive/parquet_ppd_partition.q
new file mode 100644
index 0000000..08af84f
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/parquet_ppd_partition.q
@@ -0,0 +1,9 @@
+SET hive.optimize.index.filter=true;
+SET hive.optimize.ppd=true;
+
+-- Test predicate with partitioned columns
+CREATE TABLE part1 (id int, content string) PARTITIONED BY (p string) STORED AS PARQUET;
+ALTER TABLE part1 ADD PARTITION (p='p1');
+INSERT INTO TABLE part1 PARTITION (p='p1') VALUES (1, 'a'), (2, 'b');
+SELECT * FROM part1 WHERE p='p1';
+DROP TABLE part1 PURGE;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/8f930e58/ql/src/test/queries/clientpositive/parquet_predicate_pushdown.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/parquet_predicate_pushdown.q b/ql/src/test/queries/clientpositive/parquet_predicate_pushdown.q
index 08af84f..32767e8 100644
--- a/ql/src/test/queries/clientpositive/parquet_predicate_pushdown.q
+++ b/ql/src/test/queries/clientpositive/parquet_predicate_pushdown.q
@@ -1,9 +1,292 @@
-SET hive.optimize.index.filter=true;
 SET hive.optimize.ppd=true;
 
--- Test predicate with partitioned columns
-CREATE TABLE part1 (id int, content string) PARTITIONED BY (p string) STORED AS PARQUET;
-ALTER TABLE part1 ADD PARTITION (p='p1');
-INSERT INTO TABLE part1 PARTITION (p='p1') VALUES (1, 'a'), (2, 'b');
-SELECT * FROM part1 WHERE p='p1';
-DROP TABLE part1 PURGE;
\ No newline at end of file
+-- SORT_QUERY_RESULTS
+CREATE TABLE tbl_pred(t tinyint,
+           si smallint,
+           i int,
+           b bigint,
+           f float,
+           d double,
+           bo boolean,
+           s string,
+           ts timestamp,
+           dec decimal(4,2),
+           bin binary)
+STORED AS PARQUET;
+
+CREATE TABLE staging(t tinyint,
+           si smallint,
+           i int,
+           b bigint,
+           f float,
+           d double,
+           bo boolean,
+           s string,
+           ts timestamp,
+           dec decimal(4,2),
+           bin binary)
+ROW FORMAT DELIMITED FIELDS TERMINATED BY '|'
+STORED AS TEXTFILE;
+
+LOAD DATA LOCAL INPATH '../../data/files/over1k' OVERWRITE INTO TABLE staging;
+
+INSERT INTO TABLE tbl_pred select * from staging;
+
+-- no predicate case. the explain plan should not have filter expression in table scan operator
+
+SELECT SUM(HASH(t)) FROM tbl_pred;
+
+SET hive.optimize.index.filter=true;
+SELECT SUM(HASH(t)) FROM tbl_pred;
+SET hive.optimize.index.filter=false;
+
+EXPLAIN SELECT SUM(HASH(t)) FROM tbl_pred;
+
+SET hive.optimize.index.filter=true;
+EXPLAIN SELECT SUM(HASH(t)) FROM tbl_pred;
+SET hive.optimize.index.filter=false;
+
+-- all the following queries have predicates which are pushed down to table scan operator if
+-- hive.optimize.index.filter is set to true. the explain plan should show filter expression
+-- in table scan operator.
+
+SELECT * FROM tbl_pred WHERE t<2 limit 1;
+SET hive.optimize.index.filter=true;
+SELECT * FROM tbl_pred WHERE t<2 limit 1;
+SET hive.optimize.index.filter=false;
+
+SELECT * FROM tbl_pred WHERE t>2 limit 1;
+SET hive.optimize.index.filter=true;
+SELECT * FROM tbl_pred WHERE t>2 limit 1;
+SET hive.optimize.index.filter=false;
+
+SELECT SUM(HASH(t)) FROM tbl_pred
+  WHERE t IS NOT NULL
+  AND t < 0
+  AND t > -2;
+
+SET hive.optimize.index.filter=true;
+SELECT SUM(HASH(t)) FROM tbl_pred
+  WHERE t IS NOT NULL
+  AND t < 0
+  AND t > -2;
+SET hive.optimize.index.filter=false;
+
+EXPLAIN SELECT SUM(HASH(t)) FROM tbl_pred
+  WHERE t IS NOT NULL
+  AND t < 0
+  AND t > -2;
+
+SET hive.optimize.index.filter=true;
+EXPLAIN SELECT SUM(HASH(t)) FROM tbl_pred
+  WHERE t IS NOT NULL
+  AND t < 0
+  AND t > -2;
+SET hive.optimize.index.filter=false;
+
+SELECT t, s FROM tbl_pred
+  WHERE t <=> -1
+  AND s IS NOT NULL
+  AND s LIKE 'bob%'
+  ;
+
+SET hive.optimize.index.filter=true;
+SELECT t, s FROM tbl_pred
+  WHERE t <=> -1
+  AND s IS NOT NULL
+  AND s LIKE 'bob%'
+  ;
+SET hive.optimize.index.filter=false;
+
+EXPLAIN SELECT t, s FROM tbl_pred
+  WHERE t <=> -1
+  AND s IS NOT NULL
+  AND s LIKE 'bob%'
+  ;
+
+SET hive.optimize.index.filter=true;
+EXPLAIN SELECT t, s FROM tbl_pred
+  WHERE t <=> -1
+  AND s IS NOT NULL
+  AND s LIKE 'bob%'
+  ;
+SET hive.optimize.index.filter=false;
+
+SELECT t, s FROM tbl_pred
+  WHERE s IS NOT NULL
+  AND s LIKE 'bob%'
+  AND t NOT IN (-1,-2,-3)
+  AND t BETWEEN 25 AND 30
+  SORT BY t,s;
+
+set hive.optimize.index.filter=true;
+SELECT t, s FROM tbl_pred
+  WHERE s IS NOT NULL
+  AND s LIKE 'bob%'
+  AND t NOT IN (-1,-2,-3)
+  AND t BETWEEN 25 AND 30
+  SORT BY t,s;
+set hive.optimize.index.filter=false;
+
+EXPLAIN SELECT t, s FROM tbl_pred
+  WHERE s IS NOT NULL
+  AND s LIKE 'bob%'
+  AND t NOT IN (-1,-2,-3)
+  AND t BETWEEN 25 AND 30
+  SORT BY t,s;
+
+SET hive.optimize.index.filter=true;
+EXPLAIN SELECT t, s FROM tbl_pred
+  WHERE s IS NOT NULL
+  AND s LIKE 'bob%'
+  AND t NOT IN (-1,-2,-3)
+  AND t BETWEEN 25 AND 30
+  SORT BY t,s;
+SET hive.optimize.index.filter=false;
+
+SELECT t, si, d, s FROM tbl_pred
+  WHERE d >= ROUND(9.99)
+  AND d < 12
+  AND t IS NOT NULL
+  AND s LIKE '%son'
+  AND s NOT LIKE '%car%'
+  AND t > 0
+  AND si BETWEEN 300 AND 400
+  ORDER BY s DESC
+  LIMIT 3;
+
+SET hive.optimize.index.filter=true;
+SELECT t, si, d, s FROM tbl_pred
+  WHERE d >= ROUND(9.99)
+  AND d < 12
+  AND t IS NOT NULL
+  AND s LIKE '%son'
+  AND s NOT LIKE '%car%'
+  AND t > 0
+  AND si BETWEEN 300 AND 400
+  ORDER BY s DESC
+  LIMIT 3;
+SET hive.optimize.index.filter=false;
+
+EXPLAIN SELECT t, si, d, s FROM tbl_pred
+  WHERE d >= ROUND(9.99)
+  AND d < 12
+  AND t IS NOT NULL
+  AND s LIKE '%son'
+  AND s NOT LIKE '%car%'
+  AND t > 0
+  AND si BETWEEN 300 AND 400
+  ORDER BY s DESC
+  LIMIT 3;
+
+SET hive.optimize.index.filter=true;
+EXPLAIN SELECT t, si, d, s FROM tbl_pred
+  WHERE d >= ROUND(9.99)
+  AND d < 12
+  AND t IS NOT NULL
+  AND s LIKE '%son'
+  AND s NOT LIKE '%car%'
+  AND t > 0
+  AND si BETWEEN 300 AND 400
+  ORDER BY s DESC
+  LIMIT 3;
+SET hive.optimize.index.filter=false;
+
+SELECT t, si, d, s FROM tbl_pred
+  WHERE t > 10
+  AND t <> 101
+  AND d >= ROUND(9.99)
+  AND d < 12
+  AND t IS NOT NULL
+  AND s LIKE '%son'
+  AND s NOT LIKE '%car%'
+  AND t > 0
+  AND si BETWEEN 300 AND 400
+  SORT BY s DESC
+  LIMIT 3;
+
+SET hive.optimize.index.filter=true;
+SELECT t, si, d, s FROM tbl_pred
+  WHERE t > 10
+  AND t <> 101
+  AND d >= ROUND(9.99)
+  AND d < 12
+  AND t IS NOT NULL
+  AND s LIKE '%son'
+  AND s NOT LIKE '%car%'
+  AND t > 0
+  AND si BETWEEN 300 AND 400
+  SORT BY s DESC
+  LIMIT 3;
+SET hive.optimize.index.filter=false;
+
+SET hive.optimize.index.filter=true;
+SELECT f, i, b FROM tbl_pred
+  WHERE f IS NOT NULL
+  AND f < 123.2
+  AND f > 1.92
+  AND f >= 9.99
+  AND f BETWEEN 1.92 AND 123.2
+  AND i IS NOT NULL
+  AND i < 67627
+  AND i > 60627
+  AND i >= 60626
+  AND i BETWEEN 60626 AND 67627
+  AND b IS NOT NULL
+  AND b < 4294967861
+  AND b > 4294967261
+  AND b >= 4294967260
+  AND b BETWEEN 4294967261 AND 4294967861
+  SORT BY f DESC
+  LIMIT 3;
+SET hive.optimize.index.filter=false;
+
+EXPLAIN SELECT t, si, d, s FROM tbl_pred
+  WHERE t > 10
+  AND t <> 101
+  AND d >= ROUND(9.99)
+  AND d < 12
+  AND t IS NOT NULL
+  AND s LIKE '%son'
+  AND s NOT LIKE '%car%'
+  AND t > 0
+  AND si BETWEEN 300 AND 400
+  SORT BY s DESC
+  LIMIT 3;
+
+SET hive.optimize.index.filter=true;
+EXPLAIN SELECT t, si, d, s FROM tbl_pred
+  WHERE t > 10
+  AND t <> 101
+  AND d >= ROUND(9.99)
+  AND d < 12
+  AND t IS NOT NULL
+  AND s LIKE '%son'
+  AND s NOT LIKE '%car%'
+  AND t > 0
+  AND si BETWEEN 300 AND 400
+  SORT BY s DESC
+  LIMIT 3;
+SET hive.optimize.index.filter=false;
+
+
+SET hive.optimize.index.filter=true;
+EXPLAIN SELECT f, i, b FROM tbl_pred
+  WHERE f IS NOT NULL
+  AND f < 123.2
+  AND f > 1.92
+  AND f >= 9.99
+  AND f BETWEEN 1.92 AND 123.2
+  AND i IS NOT NULL
+  AND i < 67627
+  AND i > 60627
+  AND i >= 60626
+  AND i BETWEEN 60626 AND 67627
+  AND b IS NOT NULL
+  AND b < 4294967861
+  AND b > 4294967261
+  AND b >= 4294967260
+  AND b BETWEEN 4294967261 AND 4294967861
+  SORT BY f DESC
+  LIMIT 3;
+SET hive.optimize.index.filter=false;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/8f930e58/ql/src/test/results/clientpositive/parquet_ppd_partition.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/parquet_ppd_partition.q.out b/ql/src/test/results/clientpositive/parquet_ppd_partition.q.out
new file mode 100644
index 0000000..4186618
--- /dev/null
+++ b/ql/src/test/results/clientpositive/parquet_ppd_partition.q.out
@@ -0,0 +1,47 @@
+PREHOOK: query: -- Test predicate with partitioned columns
+CREATE TABLE part1 (id int, content string) PARTITIONED BY (p string) STORED AS PARQUET
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@part1
+POSTHOOK: query: -- Test predicate with partitioned columns
+CREATE TABLE part1 (id int, content string) PARTITIONED BY (p string) STORED AS PARQUET
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@part1
+PREHOOK: query: ALTER TABLE part1 ADD PARTITION (p='p1')
+PREHOOK: type: ALTERTABLE_ADDPARTS
+PREHOOK: Output: default@part1
+POSTHOOK: query: ALTER TABLE part1 ADD PARTITION (p='p1')
+POSTHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: Output: default@part1
+POSTHOOK: Output: default@part1@p=p1
+PREHOOK: query: INSERT INTO TABLE part1 PARTITION (p='p1') VALUES (1, 'a'), (2, 'b')
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__1
+PREHOOK: Output: default@part1@p=p1
+POSTHOOK: query: INSERT INTO TABLE part1 PARTITION (p='p1') VALUES (1, 'a'), (2, 'b')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__1
+POSTHOOK: Output: default@part1@p=p1
+POSTHOOK: Lineage: part1 PARTITION(p=p1).content SIMPLE [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+POSTHOOK: Lineage: part1 PARTITION(p=p1).id EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+PREHOOK: query: SELECT * FROM part1 WHERE p='p1'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part1
+PREHOOK: Input: default@part1@p=p1
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT * FROM part1 WHERE p='p1'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part1
+POSTHOOK: Input: default@part1@p=p1
+#### A masked pattern was here ####
+1	a	p1
+2	b	p1
+PREHOOK: query: DROP TABLE part1 PURGE
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@part1
+PREHOOK: Output: default@part1
+POSTHOOK: query: DROP TABLE part1 PURGE
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@part1
+POSTHOOK: Output: default@part1

http://git-wip-us.apache.org/repos/asf/hive/blob/8f930e58/ql/src/test/results/clientpositive/parquet_predicate_pushdown.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/parquet_predicate_pushdown.q.out b/ql/src/test/results/clientpositive/parquet_predicate_pushdown.q.out
index 4186618..1dc2937 100644
--- a/ql/src/test/results/clientpositive/parquet_predicate_pushdown.q.out
+++ b/ql/src/test/results/clientpositive/parquet_predicate_pushdown.q.out
@@ -1,47 +1,1270 @@
-PREHOOK: query: -- Test predicate with partitioned columns
-CREATE TABLE part1 (id int, content string) PARTITIONED BY (p string) STORED AS PARQUET
+PREHOOK: query: -- SORT_QUERY_RESULTS
+CREATE TABLE tbl_pred(t tinyint,
+           si smallint,
+           i int,
+           b bigint,
+           f float,
+           d double,
+           bo boolean,
+           s string,
+           ts timestamp,
+           dec decimal(4,2),
+           bin binary)
+STORED AS PARQUET
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
-PREHOOK: Output: default@part1
-POSTHOOK: query: -- Test predicate with partitioned columns
-CREATE TABLE part1 (id int, content string) PARTITIONED BY (p string) STORED AS PARQUET
+PREHOOK: Output: default@tbl_pred
+POSTHOOK: query: -- SORT_QUERY_RESULTS
+CREATE TABLE tbl_pred(t tinyint,
+           si smallint,
+           i int,
+           b bigint,
+           f float,
+           d double,
+           bo boolean,
+           s string,
+           ts timestamp,
+           dec decimal(4,2),
+           bin binary)
+STORED AS PARQUET
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
-POSTHOOK: Output: default@part1
-PREHOOK: query: ALTER TABLE part1 ADD PARTITION (p='p1')
-PREHOOK: type: ALTERTABLE_ADDPARTS
-PREHOOK: Output: default@part1
-POSTHOOK: query: ALTER TABLE part1 ADD PARTITION (p='p1')
-POSTHOOK: type: ALTERTABLE_ADDPARTS
-POSTHOOK: Output: default@part1
-POSTHOOK: Output: default@part1@p=p1
-PREHOOK: query: INSERT INTO TABLE part1 PARTITION (p='p1') VALUES (1, 'a'), (2, 'b')
-PREHOOK: type: QUERY
-PREHOOK: Input: default@values__tmp__table__1
-PREHOOK: Output: default@part1@p=p1
-POSTHOOK: query: INSERT INTO TABLE part1 PARTITION (p='p1') VALUES (1, 'a'), (2, 'b')
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@values__tmp__table__1
-POSTHOOK: Output: default@part1@p=p1
-POSTHOOK: Lineage: part1 PARTITION(p=p1).content SIMPLE [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
-POSTHOOK: Lineage: part1 PARTITION(p=p1).id EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
-PREHOOK: query: SELECT * FROM part1 WHERE p='p1'
-PREHOOK: type: QUERY
-PREHOOK: Input: default@part1
-PREHOOK: Input: default@part1@p=p1
-#### A masked pattern was here ####
-POSTHOOK: query: SELECT * FROM part1 WHERE p='p1'
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@part1
-POSTHOOK: Input: default@part1@p=p1
-#### A masked pattern was here ####
-1	a	p1
-2	b	p1
-PREHOOK: query: DROP TABLE part1 PURGE
-PREHOOK: type: DROPTABLE
-PREHOOK: Input: default@part1
-PREHOOK: Output: default@part1
-POSTHOOK: query: DROP TABLE part1 PURGE
-POSTHOOK: type: DROPTABLE
-POSTHOOK: Input: default@part1
-POSTHOOK: Output: default@part1
+POSTHOOK: Output: default@tbl_pred
+PREHOOK: query: CREATE TABLE staging(t tinyint,
+           si smallint,
+           i int,
+           b bigint,
+           f float,
+           d double,
+           bo boolean,
+           s string,
+           ts timestamp,
+           dec decimal(4,2),
+           bin binary)
+ROW FORMAT DELIMITED FIELDS TERMINATED BY '|'
+STORED AS TEXTFILE
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@staging
+POSTHOOK: query: CREATE TABLE staging(t tinyint,
+           si smallint,
+           i int,
+           b bigint,
+           f float,
+           d double,
+           bo boolean,
+           s string,
+           ts timestamp,
+           dec decimal(4,2),
+           bin binary)
+ROW FORMAT DELIMITED FIELDS TERMINATED BY '|'
+STORED AS TEXTFILE
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@staging
+PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/over1k' OVERWRITE INTO TABLE staging
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: default@staging
+POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/over1k' OVERWRITE INTO TABLE staging
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: default@staging
+PREHOOK: query: INSERT INTO TABLE tbl_pred select * from staging
+PREHOOK: type: QUERY
+PREHOOK: Input: default@staging
+PREHOOK: Output: default@tbl_pred
+POSTHOOK: query: INSERT INTO TABLE tbl_pred select * from staging
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@staging
+POSTHOOK: Output: default@tbl_pred
+POSTHOOK: Lineage: tbl_pred.b SIMPLE [(staging)staging.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: tbl_pred.bin SIMPLE [(staging)staging.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: tbl_pred.bo SIMPLE [(staging)staging.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: tbl_pred.d SIMPLE [(staging)staging.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: tbl_pred.dec SIMPLE [(staging)staging.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: tbl_pred.f SIMPLE [(staging)staging.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: tbl_pred.i SIMPLE [(staging)staging.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: tbl_pred.s SIMPLE [(staging)staging.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: tbl_pred.si SIMPLE [(staging)staging.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: tbl_pred.t SIMPLE [(staging)staging.FieldSchema(name:t, type:tinyint, comment:null), ]
+POSTHOOK: Lineage: tbl_pred.ts SIMPLE [(staging)staging.FieldSchema(name:ts, type:timestamp, comment:null), ]
+PREHOOK: query: -- no predicate case. the explain plan should not have filter expression in table scan operator
+
+SELECT SUM(HASH(t)) FROM tbl_pred
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_pred
+#### A masked pattern was here ####
+POSTHOOK: query: -- no predicate case. the explain plan should not have filter expression in table scan operator
+
+SELECT SUM(HASH(t)) FROM tbl_pred
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_pred
+#### A masked pattern was here ####
+62430
+PREHOOK: query: SELECT SUM(HASH(t)) FROM tbl_pred
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_pred
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT SUM(HASH(t)) FROM tbl_pred
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_pred
+#### A masked pattern was here ####
+62430
+PREHOOK: query: EXPLAIN SELECT SUM(HASH(t)) FROM tbl_pred
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN SELECT SUM(HASH(t)) FROM tbl_pred
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: tbl_pred
+            Statistics: Num rows: 1049 Data size: 11539 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: hash(t) (type: int)
+              outputColumnNames: _col0
+              Statistics: Num rows: 1049 Data size: 11539 Basic stats: COMPLETE Column stats: NONE
+              Group By Operator
+                aggregations: sum(_col0)
+                mode: hash
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col0 (type: bigint)
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations: sum(VALUE._col0)
+          mode: mergepartial
+          outputColumnNames: _col0
+          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+          File Output Operator
+            compressed: false
+            Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+            table:
+                input format: org.apache.hadoop.mapred.TextInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: EXPLAIN SELECT SUM(HASH(t)) FROM tbl_pred
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN SELECT SUM(HASH(t)) FROM tbl_pred
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: tbl_pred
+            Statistics: Num rows: 1049 Data size: 11539 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: hash(t) (type: int)
+              outputColumnNames: _col0
+              Statistics: Num rows: 1049 Data size: 11539 Basic stats: COMPLETE Column stats: NONE
+              Group By Operator
+                aggregations: sum(_col0)
+                mode: hash
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col0 (type: bigint)
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations: sum(VALUE._col0)
+          mode: mergepartial
+          outputColumnNames: _col0
+          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+          File Output Operator
+            compressed: false
+            Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+            table:
+                input format: org.apache.hadoop.mapred.TextInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: -- all the following queries have predicates which are pushed down to table scan operator if
+-- hive.optimize.index.filter is set to true. the explain plan should show filter expression
+-- in table scan operator.
+
+SELECT * FROM tbl_pred WHERE t<2 limit 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_pred
+#### A masked pattern was here ####
+POSTHOOK: query: -- all the following queries have predicates which are pushed down to table scan operator if
+-- hive.optimize.index.filter is set to true. the explain plan should show filter expression
+-- in table scan operator.
+
+SELECT * FROM tbl_pred WHERE t<2 limit 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_pred
+#### A masked pattern was here ####
+-3	467	65575	4294967437	81.64	23.53	true	tom hernandez	2013-03-01 09:11:58.703188	32.85	study skills
+PREHOOK: query: SELECT * FROM tbl_pred WHERE t<2 limit 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_pred
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT * FROM tbl_pred WHERE t<2 limit 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_pred
+#### A masked pattern was here ####
+-3	467	65575	4294967437	81.64	23.53	true	tom hernandez	2013-03-01 09:11:58.703188	32.85	study skills
+PREHOOK: query: SELECT * FROM tbl_pred WHERE t>2 limit 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_pred
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT * FROM tbl_pred WHERE t>2 limit 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_pred
+#### A masked pattern was here ####
+124	336	65664	4294967435	74.72	42.47	true	bob davidson	2013-03-01 09:11:58.703302	45.4	yard duty
+PREHOOK: query: SELECT * FROM tbl_pred WHERE t>2 limit 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_pred
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT * FROM tbl_pred WHERE t>2 limit 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_pred
+#### A masked pattern was here ####
+124	336	65664	4294967435	74.72	42.47	true	bob davidson	2013-03-01 09:11:58.703302	45.4	yard duty
+PREHOOK: query: SELECT SUM(HASH(t)) FROM tbl_pred
+  WHERE t IS NOT NULL
+  AND t < 0
+  AND t > -2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_pred
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT SUM(HASH(t)) FROM tbl_pred
+  WHERE t IS NOT NULL
+  AND t < 0
+  AND t > -2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_pred
+#### A masked pattern was here ####
+-8
+PREHOOK: query: SELECT SUM(HASH(t)) FROM tbl_pred
+  WHERE t IS NOT NULL
+  AND t < 0
+  AND t > -2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_pred
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT SUM(HASH(t)) FROM tbl_pred
+  WHERE t IS NOT NULL
+  AND t < 0
+  AND t > -2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_pred
+#### A masked pattern was here ####
+-8
+PREHOOK: query: EXPLAIN SELECT SUM(HASH(t)) FROM tbl_pred
+  WHERE t IS NOT NULL
+  AND t < 0
+  AND t > -2
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN SELECT SUM(HASH(t)) FROM tbl_pred
+  WHERE t IS NOT NULL
+  AND t < 0
+  AND t > -2
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: tbl_pred
+            Statistics: Num rows: 1049 Data size: 11539 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: ((t < 0) and (UDFToInteger(t) > -2)) (type: boolean)
+              Statistics: Num rows: 116 Data size: 1276 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: hash(t) (type: int)
+                outputColumnNames: _col0
+                Statistics: Num rows: 116 Data size: 1276 Basic stats: COMPLETE Column stats: NONE
+                Group By Operator
+                  aggregations: sum(_col0)
+                  mode: hash
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col0 (type: bigint)
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations: sum(VALUE._col0)
+          mode: mergepartial
+          outputColumnNames: _col0
+          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+          File Output Operator
+            compressed: false
+            Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+            table:
+                input format: org.apache.hadoop.mapred.TextInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: EXPLAIN SELECT SUM(HASH(t)) FROM tbl_pred
+  WHERE t IS NOT NULL
+  AND t < 0
+  AND t > -2
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN SELECT SUM(HASH(t)) FROM tbl_pred
+  WHERE t IS NOT NULL
+  AND t < 0
+  AND t > -2
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: tbl_pred
+            filterExpr: ((t < 0) and (UDFToInteger(t) > -2)) (type: boolean)
+            Statistics: Num rows: 1049 Data size: 11539 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: ((t < 0) and (UDFToInteger(t) > -2)) (type: boolean)
+              Statistics: Num rows: 116 Data size: 1276 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: hash(t) (type: int)
+                outputColumnNames: _col0
+                Statistics: Num rows: 116 Data size: 1276 Basic stats: COMPLETE Column stats: NONE
+                Group By Operator
+                  aggregations: sum(_col0)
+                  mode: hash
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col0 (type: bigint)
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations: sum(VALUE._col0)
+          mode: mergepartial
+          outputColumnNames: _col0
+          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+          File Output Operator
+            compressed: false
+            Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+            table:
+                input format: org.apache.hadoop.mapred.TextInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT t, s FROM tbl_pred
+  WHERE t <=> -1
+  AND s IS NOT NULL
+  AND s LIKE 'bob%'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_pred
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT t, s FROM tbl_pred
+  WHERE t <=> -1
+  AND s IS NOT NULL
+  AND s LIKE 'bob%'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_pred
+#### A masked pattern was here ####
+-1	bob laertes
+-1	bob young
+PREHOOK: query: SELECT t, s FROM tbl_pred
+  WHERE t <=> -1
+  AND s IS NOT NULL
+  AND s LIKE 'bob%'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_pred
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT t, s FROM tbl_pred
+  WHERE t <=> -1
+  AND s IS NOT NULL
+  AND s LIKE 'bob%'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_pred
+#### A masked pattern was here ####
+-1	bob laertes
+-1	bob young
+PREHOOK: query: EXPLAIN SELECT t, s FROM tbl_pred
+  WHERE t <=> -1
+  AND s IS NOT NULL
+  AND s LIKE 'bob%'
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN SELECT t, s FROM tbl_pred
+  WHERE t <=> -1
+  AND s IS NOT NULL
+  AND s LIKE 'bob%'
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: tbl_pred
+            Statistics: Num rows: 1049 Data size: 11539 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: (((t = -1) and s is not null) and (s like 'bob%')) (type: boolean)
+              Statistics: Num rows: 131 Data size: 1441 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: -1 (type: tinyint), s (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 131 Data size: 1441 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 131 Data size: 1441 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: EXPLAIN SELECT t, s FROM tbl_pred
+  WHERE t <=> -1
+  AND s IS NOT NULL
+  AND s LIKE 'bob%'
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN SELECT t, s FROM tbl_pred
+  WHERE t <=> -1
+  AND s IS NOT NULL
+  AND s LIKE 'bob%'
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: tbl_pred
+            filterExpr: (((t = -1) and s is not null) and (s like 'bob%')) (type: boolean)
+            Statistics: Num rows: 1049 Data size: 11539 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: (((t = -1) and s is not null) and (s like 'bob%')) (type: boolean)
+              Statistics: Num rows: 131 Data size: 1441 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: -1 (type: tinyint), s (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 131 Data size: 1441 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 131 Data size: 1441 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT t, s FROM tbl_pred
+  WHERE s IS NOT NULL
+  AND s LIKE 'bob%'
+  AND t NOT IN (-1,-2,-3)
+  AND t BETWEEN 25 AND 30
+  SORT BY t,s
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_pred
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT t, s FROM tbl_pred
+  WHERE s IS NOT NULL
+  AND s LIKE 'bob%'
+  AND t NOT IN (-1,-2,-3)
+  AND t BETWEEN 25 AND 30
+  SORT BY t,s
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_pred
+#### A masked pattern was here ####
+26	bob ovid
+26	bob quirinius
+27	bob ovid
+PREHOOK: query: SELECT t, s FROM tbl_pred
+  WHERE s IS NOT NULL
+  AND s LIKE 'bob%'
+  AND t NOT IN (-1,-2,-3)
+  AND t BETWEEN 25 AND 30
+  SORT BY t,s
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_pred
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT t, s FROM tbl_pred
+  WHERE s IS NOT NULL
+  AND s LIKE 'bob%'
+  AND t NOT IN (-1,-2,-3)
+  AND t BETWEEN 25 AND 30
+  SORT BY t,s
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_pred
+#### A masked pattern was here ####
+26	bob ovid
+26	bob quirinius
+27	bob ovid
+PREHOOK: query: EXPLAIN SELECT t, s FROM tbl_pred
+  WHERE s IS NOT NULL
+  AND s LIKE 'bob%'
+  AND t NOT IN (-1,-2,-3)
+  AND t BETWEEN 25 AND 30
+  SORT BY t,s
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN SELECT t, s FROM tbl_pred
+  WHERE s IS NOT NULL
+  AND s LIKE 'bob%'
+  AND t NOT IN (-1,-2,-3)
+  AND t BETWEEN 25 AND 30
+  SORT BY t,s
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: tbl_pred
+            Statistics: Num rows: 1049 Data size: 11539 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: (((s is not null and (s like 'bob%')) and (not (t) IN (-1, -2, -3))) and t BETWEEN 25 AND 30) (type: boolean)
+              Statistics: Num rows: 65 Data size: 715 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: t (type: tinyint), s (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 65 Data size: 715 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col0 (type: tinyint), _col1 (type: string)
+                  sort order: ++
+                  Statistics: Num rows: 65 Data size: 715 Basic stats: COMPLETE Column stats: NONE
+      Reduce Operator Tree:
+        Select Operator
+          expressions: KEY.reducesinkkey0 (type: tinyint), KEY.reducesinkkey1 (type: string)
+          outputColumnNames: _col0, _col1
+          Statistics: Num rows: 65 Data size: 715 Basic stats: COMPLETE Column stats: NONE
+          File Output Operator
+            compressed: false
+            Statistics: Num rows: 65 Data size: 715 Basic stats: COMPLETE Column stats: NONE
+            table:
+                input format: org.apache.hadoop.mapred.TextInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: EXPLAIN SELECT t, s FROM tbl_pred
+  WHERE s IS NOT NULL
+  AND s LIKE 'bob%'
+  AND t NOT IN (-1,-2,-3)
+  AND t BETWEEN 25 AND 30
+  SORT BY t,s
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN SELECT t, s FROM tbl_pred
+  WHERE s IS NOT NULL
+  AND s LIKE 'bob%'
+  AND t NOT IN (-1,-2,-3)
+  AND t BETWEEN 25 AND 30
+  SORT BY t,s
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: tbl_pred
+            filterExpr: (((s is not null and (s like 'bob%')) and (not (t) IN (-1, -2, -3))) and t BETWEEN 25 AND 30) (type: boolean)
+            Statistics: Num rows: 1049 Data size: 11539 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: (((s is not null and (s like 'bob%')) and (not (t) IN (-1, -2, -3))) and t BETWEEN 25 AND 30) (type: boolean)
+              Statistics: Num rows: 65 Data size: 715 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: t (type: tinyint), s (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 65 Data size: 715 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col0 (type: tinyint), _col1 (type: string)
+                  sort order: ++
+                  Statistics: Num rows: 65 Data size: 715 Basic stats: COMPLETE Column stats: NONE
+      Reduce Operator Tree:
+        Select Operator
+          expressions: KEY.reducesinkkey0 (type: tinyint), KEY.reducesinkkey1 (type: string)
+          outputColumnNames: _col0, _col1
+          Statistics: Num rows: 65 Data size: 715 Basic stats: COMPLETE Column stats: NONE
+          File Output Operator
+            compressed: false
+            Statistics: Num rows: 65 Data size: 715 Basic stats: COMPLETE Column stats: NONE
+            table:
+                input format: org.apache.hadoop.mapred.TextInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT t, si, d, s FROM tbl_pred
+  WHERE d >= ROUND(9.99)
+  AND d < 12
+  AND t IS NOT NULL
+  AND s LIKE '%son'
+  AND s NOT LIKE '%car%'
+  AND t > 0
+  AND si BETWEEN 300 AND 400
+  ORDER BY s DESC
+  LIMIT 3
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_pred
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT t, si, d, s FROM tbl_pred
+  WHERE d >= ROUND(9.99)
+  AND d < 12
+  AND t IS NOT NULL
+  AND s LIKE '%son'
+  AND s NOT LIKE '%car%'
+  AND t > 0
+  AND si BETWEEN 300 AND 400
+  ORDER BY s DESC
+  LIMIT 3
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_pred
+#### A masked pattern was here ####
+101	327	11.48	gabriella ellison
+15	334	11.12	jessica robinson
+7	320	11.54	bob ellison
+PREHOOK: query: SELECT t, si, d, s FROM tbl_pred
+  WHERE d >= ROUND(9.99)
+  AND d < 12
+  AND t IS NOT NULL
+  AND s LIKE '%son'
+  AND s NOT LIKE '%car%'
+  AND t > 0
+  AND si BETWEEN 300 AND 400
+  ORDER BY s DESC
+  LIMIT 3
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_pred
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT t, si, d, s FROM tbl_pred
+  WHERE d >= ROUND(9.99)
+  AND d < 12
+  AND t IS NOT NULL
+  AND s LIKE '%son'
+  AND s NOT LIKE '%car%'
+  AND t > 0
+  AND si BETWEEN 300 AND 400
+  ORDER BY s DESC
+  LIMIT 3
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_pred
+#### A masked pattern was here ####
+101	327	11.48	gabriella ellison
+15	334	11.12	jessica robinson
+7	320	11.54	bob ellison
+PREHOOK: query: EXPLAIN SELECT t, si, d, s FROM tbl_pred
+  WHERE d >= ROUND(9.99)
+  AND d < 12
+  AND t IS NOT NULL
+  AND s LIKE '%son'
+  AND s NOT LIKE '%car%'
+  AND t > 0
+  AND si BETWEEN 300 AND 400
+  ORDER BY s DESC
+  LIMIT 3
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN SELECT t, si, d, s FROM tbl_pred
+  WHERE d >= ROUND(9.99)
+  AND d < 12
+  AND t IS NOT NULL
+  AND s LIKE '%son'
+  AND s NOT LIKE '%car%'
+  AND t > 0
+  AND si BETWEEN 300 AND 400
+  ORDER BY s DESC
+  LIMIT 3
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: tbl_pred
+            Statistics: Num rows: 1049 Data size: 11539 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: ((((((d >= 10.0) and (d < 12.0)) and (s like '%son')) and (not (s like '%car%'))) and (t > 0)) and si BETWEEN 300 AND 400) (type: boolean)
+              Statistics: Num rows: 4 Data size: 44 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: t (type: tinyint), si (type: smallint), d (type: double), s (type: string)
+                outputColumnNames: _col0, _col1, _col2, _col3
+                Statistics: Num rows: 4 Data size: 44 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col3 (type: string)
+                  sort order: -
+                  Statistics: Num rows: 4 Data size: 44 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: double)
+      Reduce Operator Tree:
+        Select Operator
+          expressions: VALUE._col0 (type: tinyint), VALUE._col1 (type: smallint), VALUE._col2 (type: double), KEY.reducesinkkey0 (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3
+          Statistics: Num rows: 4 Data size: 44 Basic stats: COMPLETE Column stats: NONE
+          Limit
+            Number of rows: 3
+            Statistics: Num rows: 3 Data size: 33 Basic stats: COMPLETE Column stats: NONE
+            File Output Operator
+              compressed: false
+              Statistics: Num rows: 3 Data size: 33 Basic stats: COMPLETE Column stats: NONE
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: 3
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: EXPLAIN SELECT t, si, d, s FROM tbl_pred
+  WHERE d >= ROUND(9.99)
+  AND d < 12
+  AND t IS NOT NULL
+  AND s LIKE '%son'
+  AND s NOT LIKE '%car%'
+  AND t > 0
+  AND si BETWEEN 300 AND 400
+  ORDER BY s DESC
+  LIMIT 3
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN SELECT t, si, d, s FROM tbl_pred
+  WHERE d >= ROUND(9.99)
+  AND d < 12
+  AND t IS NOT NULL
+  AND s LIKE '%son'
+  AND s NOT LIKE '%car%'
+  AND t > 0
+  AND si BETWEEN 300 AND 400
+  ORDER BY s DESC
+  LIMIT 3
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: tbl_pred
+            filterExpr: ((((((d >= 10.0) and (d < 12.0)) and (s like '%son')) and (not (s like '%car%'))) and (t > 0)) and si BETWEEN 300 AND 400) (type: boolean)
+            Statistics: Num rows: 1049 Data size: 11539 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: ((((((d >= 10.0) and (d < 12.0)) and (s like '%son')) and (not (s like '%car%'))) and (t > 0)) and si BETWEEN 300 AND 400) (type: boolean)
+              Statistics: Num rows: 4 Data size: 44 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: t (type: tinyint), si (type: smallint), d (type: double), s (type: string)
+                outputColumnNames: _col0, _col1, _col2, _col3
+                Statistics: Num rows: 4 Data size: 44 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col3 (type: string)
+                  sort order: -
+                  Statistics: Num rows: 4 Data size: 44 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: double)
+      Reduce Operator Tree:
+        Select Operator
+          expressions: VALUE._col0 (type: tinyint), VALUE._col1 (type: smallint), VALUE._col2 (type: double), KEY.reducesinkkey0 (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3
+          Statistics: Num rows: 4 Data size: 44 Basic stats: COMPLETE Column stats: NONE
+          Limit
+            Number of rows: 3
+            Statistics: Num rows: 3 Data size: 33 Basic stats: COMPLETE Column stats: NONE
+            File Output Operator
+              compressed: false
+              Statistics: Num rows: 3 Data size: 33 Basic stats: COMPLETE Column stats: NONE
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: 3
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT t, si, d, s FROM tbl_pred
+  WHERE t > 10
+  AND t <> 101
+  AND d >= ROUND(9.99)
+  AND d < 12
+  AND t IS NOT NULL
+  AND s LIKE '%son'
+  AND s NOT LIKE '%car%'
+  AND t > 0
+  AND si BETWEEN 300 AND 400
+  SORT BY s DESC
+  LIMIT 3
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_pred
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT t, si, d, s FROM tbl_pred
+  WHERE t > 10
+  AND t <> 101
+  AND d >= ROUND(9.99)
+  AND d < 12
+  AND t IS NOT NULL
+  AND s LIKE '%son'
+  AND s NOT LIKE '%car%'
+  AND t > 0
+  AND si BETWEEN 300 AND 400
+  SORT BY s DESC
+  LIMIT 3
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_pred
+#### A masked pattern was here ####
+15	334	11.12	jessica robinson
+PREHOOK: query: SELECT t, si, d, s FROM tbl_pred
+  WHERE t > 10
+  AND t <> 101
+  AND d >= ROUND(9.99)
+  AND d < 12
+  AND t IS NOT NULL
+  AND s LIKE '%son'
+  AND s NOT LIKE '%car%'
+  AND t > 0
+  AND si BETWEEN 300 AND 400
+  SORT BY s DESC
+  LIMIT 3
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_pred
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT t, si, d, s FROM tbl_pred
+  WHERE t > 10
+  AND t <> 101
+  AND d >= ROUND(9.99)
+  AND d < 12
+  AND t IS NOT NULL
+  AND s LIKE '%son'
+  AND s NOT LIKE '%car%'
+  AND t > 0
+  AND si BETWEEN 300 AND 400
+  SORT BY s DESC
+  LIMIT 3
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_pred
+#### A masked pattern was here ####
+15	334	11.12	jessica robinson
+PREHOOK: query: SELECT f, i, b FROM tbl_pred
+  WHERE f IS NOT NULL
+  AND f < 123.2
+  AND f > 1.92
+  AND f >= 9.99
+  AND f BETWEEN 1.92 AND 123.2
+  AND i IS NOT NULL
+  AND i < 67627
+  AND i > 60627
+  AND i >= 60626
+  AND i BETWEEN 60626 AND 67627
+  AND b IS NOT NULL
+  AND b < 4294967861
+  AND b > 4294967261
+  AND b >= 4294967260
+  AND b BETWEEN 4294967261 AND 4294967861
+  SORT BY f DESC
+  LIMIT 3
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_pred
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT f, i, b FROM tbl_pred
+  WHERE f IS NOT NULL
+  AND f < 123.2
+  AND f > 1.92
+  AND f >= 9.99
+  AND f BETWEEN 1.92 AND 123.2
+  AND i IS NOT NULL
+  AND i < 67627
+  AND i > 60627
+  AND i >= 60626
+  AND i BETWEEN 60626 AND 67627
+  AND b IS NOT NULL
+  AND b < 4294967861
+  AND b > 4294967261
+  AND b >= 4294967260
+  AND b BETWEEN 4294967261 AND 4294967861
+  SORT BY f DESC
+  LIMIT 3
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_pred
+#### A masked pattern was here ####
+99.68	65658	4294967503
+99.91	65763	4294967324
+99.92	65661	4294967404
+PREHOOK: query: EXPLAIN SELECT t, si, d, s FROM tbl_pred
+  WHERE t > 10
+  AND t <> 101
+  AND d >= ROUND(9.99)
+  AND d < 12
+  AND t IS NOT NULL
+  AND s LIKE '%son'
+  AND s NOT LIKE '%car%'
+  AND t > 0
+  AND si BETWEEN 300 AND 400
+  SORT BY s DESC
+  LIMIT 3
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN SELECT t, si, d, s FROM tbl_pred
+  WHERE t > 10
+  AND t <> 101
+  AND d >= ROUND(9.99)
+  AND d < 12
+  AND t IS NOT NULL
+  AND s LIKE '%son'
+  AND s NOT LIKE '%car%'
+  AND t > 0
+  AND si BETWEEN 300 AND 400
+  SORT BY s DESC
+  LIMIT 3
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1
+  Stage-0 depends on stages: Stage-2
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: tbl_pred
+            Statistics: Num rows: 1049 Data size: 11539 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: ((((((((t > 10) and (t <> 101)) and (d >= 10.0)) and (d < 12.0)) and (s like '%son')) and (not (s like '%car%'))) and (t > 0)) and si BETWEEN 300 AND 400) (type: boolean)
+              Statistics: Num rows: 1 Data size: 11 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: t (type: tinyint), si (type: smallint), d (type: double), s (type: string)
+                outputColumnNames: _col0, _col1, _col2, _col3
+                Statistics: Num rows: 1 Data size: 11 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col3 (type: string)
+                  sort order: -
+                  Statistics: Num rows: 1 Data size: 11 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: double)
+      Reduce Operator Tree:
+        Select Operator
+          expressions: VALUE._col0 (type: tinyint), VALUE._col1 (type: smallint), VALUE._col2 (type: double), KEY.reducesinkkey0 (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3
+          Statistics: Num rows: 1 Data size: 11 Basic stats: COMPLETE Column stats: NONE
+          Limit
+            Number of rows: 3
+            Statistics: Num rows: 1 Data size: 11 Basic stats: COMPLETE Column stats: NONE
+            File Output Operator
+              compressed: false
+              table:
+                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-2
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            Reduce Output Operator
+              key expressions: _col3 (type: string)
+              sort order: -
+              Statistics: Num rows: 1 Data size: 11 Basic stats: COMPLETE Column stats: NONE
+              value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: double)
+      Reduce Operator Tree:
+        Select Operator
+          expressions: VALUE._col0 (type: tinyint), VALUE._col1 (type: smallint), VALUE._col2 (type: double), KEY.reducesinkkey0 (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3
+          Statistics: Num rows: 1 Data size: 11 Basic stats: COMPLETE Column stats: NONE
+          Limit
+            Number of rows: 3
+            Statistics: Num rows: 1 Data size: 11 Basic stats: COMPLETE Column stats: NONE
+            File Output Operator
+              compressed: false
+              Statistics: Num rows: 1 Data size: 11 Basic stats: COMPLETE Column stats: NONE
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: 3
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: EXPLAIN SELECT t, si, d, s FROM tbl_pred
+  WHERE t > 10
+  AND t <> 101
+  AND d >= ROUND(9.99)
+  AND d < 12
+  AND t IS NOT NULL
+  AND s LIKE '%son'
+  AND s NOT LIKE '%car%'
+  AND t > 0
+  AND si BETWEEN 300 AND 400
+  SORT BY s DESC
+  LIMIT 3
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN SELECT t, si, d, s FROM tbl_pred
+  WHERE t > 10
+  AND t <> 101
+  AND d >= ROUND(9.99)
+  AND d < 12
+  AND t IS NOT NULL
+  AND s LIKE '%son'
+  AND s NOT LIKE '%car%'
+  AND t > 0
+  AND si BETWEEN 300 AND 400
+  SORT BY s DESC
+  LIMIT 3
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1
+  Stage-0 depends on stages: Stage-2
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: tbl_pred
+            filterExpr: ((((((((t > 10) and (t <> 101)) and (d >= 10.0)) and (d < 12.0)) and (s like '%son')) and (not (s like '%car%'))) and (t > 0)) and si BETWEEN 300 AND 400) (type: boolean)
+            Statistics: Num rows: 1049 Data size: 11539 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: ((((((((t > 10) and (t <> 101)) and (d >= 10.0)) and (d < 12.0)) and (s like '%son')) and (not (s like '%car%'))) and (t > 0)) and si BETWEEN 300 AND 400) (type: boolean)
+              Statistics: Num rows: 1 Data size: 11 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: t (type: tinyint), si (type: smallint), d (type: double), s (type: string)
+                outputColumnNames: _col0, _col1, _col2, _col3
+                Statistics: Num rows: 1 Data size: 11 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col3 (type: string)
+                  sort order: -
+                  Statistics: Num rows: 1 Data size: 11 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: double)
+      Reduce Operator Tree:
+        Select Operator
+          expressions: VALUE._col0 (type: tinyint), VALUE._col1 (type: smallint), VALUE._col2 (type: double), KEY.reducesinkkey0 (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3
+          Statistics: Num rows: 1 Data size: 11 Basic stats: COMPLETE Column stats: NONE
+          Limit
+            Number of rows: 3
+            Statistics: Num rows: 1 Data size: 11 Basic stats: COMPLETE Column stats: NONE
+            File Output Operator
+              compressed: false
+              table:
+                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-2
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            Reduce Output Operator
+              key expressions: _col3 (type: string)
+              sort order: -
+              Statistics: Num rows: 1 Data size: 11 Basic stats: COMPLETE Column stats: NONE
+              value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: double)
+      Reduce Operator Tree:
+        Select Operator
+          expressions: VALUE._col0 (type: tinyint), VALUE._col1 (type: smallint), VALUE._col2 (type: double), KEY.reducesinkkey0 (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3
+          Statistics: Num rows: 1 Data size: 11 Basic stats: COMPLETE Column stats: NONE
+          Limit
+            Number of rows: 3
+            Statistics: Num rows: 1 Data size: 11 Basic stats: COMPLETE Column stats: NONE
+            File Output Operator
+              compressed: false
+              Statistics: Num rows: 1 Data size: 11 Basic stats: COMPLETE Column stats: NONE
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: 3
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: EXPLAIN SELECT f, i, b FROM tbl_pred
+  WHERE f IS NOT NULL
+  AND f < 123.2
+  AND f > 1.92
+  AND f >= 9.99
+  AND f BETWEEN 1.92 AND 123.2
+  AND i IS NOT NULL
+  AND i < 67627
+  AND i > 60627
+  AND i >= 60626
+  AND i BETWEEN 60626 AND 67627
+  AND b IS NOT NULL
+  AND b < 4294967861
+  AND b > 4294967261
+  AND b >= 4294967260
+  AND b BETWEEN 4294967261 AND 4294967861
+  SORT BY f DESC
+  LIMIT 3
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN SELECT f, i, b FROM tbl_pred
+  WHERE f IS NOT NULL
+  AND f < 123.2
+  AND f > 1.92
+  AND f >= 9.99
+  AND f BETWEEN 1.92 AND 123.2
+  AND i IS NOT NULL
+  AND i < 67627
+  AND i > 60627
+  AND i >= 60626
+  AND i BETWEEN 60626 AND 67627
+  AND b IS NOT NULL
+  AND b < 4294967861
+  AND b > 4294967261
+  AND b >= 4294967260
+  AND b BETWEEN 4294967261 AND 4294967861
+  SORT BY f DESC
+  LIMIT 3
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1
+  Stage-0 depends on stages: Stage-2
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: tbl_pred
+            filterExpr: ((((((((((((f < 123.2) and (f > 1.92)) and (f >= 9.99)) and f BETWEEN 1.92 AND 123.2) and (i < 67627)) and (i > 60627)) and (i >= 60626)) and i BETWEEN 60626 AND 67627) and (b < 4294967861)) and (b > 4294967261)) and (b >= 4294967260)) and b BETWEEN 4294967261 AND 4294967861) (type: boolean)
+            Statistics: Num rows: 1049 Data size: 11539 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: ((((((((((((f < 123.2) and (f > 1.92)) and (f >= 9.99)) and f BETWEEN 1.92 AND 123.2) and (i < 67627)) and (i > 60627)) and (i >= 60626)) and i BETWEEN 60626 AND 67627) and (b < 4294967861)) and (b > 4294967261)) and (b >= 4294967260)) and b BETWEEN 4294967261 AND 4294967861) (type: boolean)
+              Statistics: Num rows: 1 Data size: 11 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: f (type: float), i (type: int), b (type: bigint)
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 11 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col0 (type: float)
+                  sort order: -
+                  Statistics: Num rows: 1 Data size: 11 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col1 (type: int), _col2 (type: bigint)
+      Reduce Operator Tree:
+        Select Operator
+          expressions: KEY.reducesinkkey0 (type: float), VALUE._col0 (type: int), VALUE._col1 (type: bigint)
+          outputColumnNames: _col0, _col1, _col2
+          Statistics: Num rows: 1 Data size: 11 Basic stats: COMPLETE Column stats: NONE
+          Limit
+            Number of rows: 3
+            Statistics: Num rows: 1 Data size: 11 Basic stats: COMPLETE Column stats: NONE
+            File Output Operator
+              compressed: false
+              table:
+                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-2
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            Reduce Output Operator
+              key expressions: _col0 (type: float)
+              sort order: -
+              Statistics: Num rows: 1 Data size: 11 Basic stats: COMPLETE Column stats: NONE
+              value expressions: _col1 (type: int), _col2 (type: bigint)
+      Reduce Operator Tree:
+        Select Operator
+          expressions: KEY.reducesinkkey0 (type: float), VALUE._col0 (type: int), VALUE._col1 (type: bigint)
+          outputColumnNames: _col0, _col1, _col2
+          Statistics: Num rows: 1 Data size: 11 Basic stats: COMPLETE Column stats: NONE
+          Limit
+            Number of rows: 3
+            Statistics: Num rows: 1 Data size: 11 Basic stats: COMPLETE Column stats: NONE
+            File Output Operator
+              compressed: false
+              Statistics: Num rows: 1 Data size: 11 Basic stats: COMPLETE Column stats: NONE
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: 3
+      Processor Tree:
+        ListSink
+