You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by jd...@apache.org on 2017/02/16 21:52:54 UTC

hive git commit: HIVE-15902: Select query involving date throwing Hive 2 Internal error: unsupported conversion from type: date (Jason Dere, reviewed by Matt McCline)

Repository: hive
Updated Branches:
  refs/heads/master e49a07426 -> f3790ce26


HIVE-15902: Select query involving date throwing Hive 2 Internal error: unsupported conversion from type: date (Jason Dere, reviewed by Matt McCline)


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

Branch: refs/heads/master
Commit: f3790ce2604621ac2512053b50eaca1ad053f178
Parents: e49a074
Author: Jason Dere <jd...@hortonworks.com>
Authored: Thu Feb 16 13:52:29 2017 -0800
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Thu Feb 16 13:52:29 2017 -0800

----------------------------------------------------------------------
 .../test/resources/testconfiguration.properties |   1 +
 .../FilterColumnBetweenDynamicValue.txt         |   9 +-
 .../ql/exec/vector/VectorizationContext.java    |   2 +-
 .../vectorized_dynamic_semijoin_reduction2.q    |  50 ++
 ...vectorized_dynamic_semijoin_reduction2.q.out | 772 +++++++++++++++++++
 .../apache/hadoop/hive/tools/GenVectorCode.java |  18 +
 6 files changed, 847 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/f3790ce2/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index 2c53047..e8db920 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -610,6 +610,7 @@ minillaplocal.query.files=acid_globallimit.q,\
   vectorization_short_regress.q,\
   vectorized_dynamic_partition_pruning.q,\
   vectorized_dynamic_semijoin_reduction.q,\
+  vectorized_dynamic_semijoin_reduction2.q,\
   vectorized_ptf.q,\
   windowing.q,\
   windowing_gby.q,\

http://git-wip-us.apache.org/repos/asf/hive/blob/f3790ce2/ql/src/gen/vectorization/ExpressionTemplates/FilterColumnBetweenDynamicValue.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/FilterColumnBetweenDynamicValue.txt b/ql/src/gen/vectorization/ExpressionTemplates/FilterColumnBetweenDynamicValue.txt
index 97ab7aa..1aee9b3 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/FilterColumnBetweenDynamicValue.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/FilterColumnBetweenDynamicValue.txt
@@ -22,6 +22,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.Filter<TypeName>ColumnBetween;
 import org.apache.hadoop.hive.ql.plan.DynamicValue;
+import org.apache.hadoop.hive.serde2.io.DateWritable;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -79,12 +80,12 @@ public class <ClassName> extends Filter<TypeName>ColumnBetween {
       if (lVal == null || rVal == null) {
         isLeftOrRightNull = true;
       } else {
-        <VectorType> min = PrimitiveObjectInspectorUtils.<GetPrimitiveMethod>(
-            lVal, leftDynamicValue.getObjectInspector())<GetValueMethod>;
+        <VectorType> min = <ConversionMethod>(PrimitiveObjectInspectorUtils.<GetPrimitiveMethod>(
+            lVal, leftDynamicValue.getObjectInspector())<GetValueMethod>);
         setLeftValue(min);
 
-        <VectorType> max = PrimitiveObjectInspectorUtils.<GetPrimitiveMethod>(
-            rVal, rightDynamicValue.getObjectInspector())<GetValueMethod>;
+        <VectorType> max = <ConversionMethod>(PrimitiveObjectInspectorUtils.<GetPrimitiveMethod>(
+            rVal, rightDynamicValue.getObjectInspector())<GetValueMethod>);
         setRightValue(max);
       }
       initialized = true;

http://git-wip-us.apache.org/repos/asf/hive/blob/f3790ce2/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
index 4802489..8164684 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
@@ -2240,7 +2240,7 @@ public class VectorizationContext {
       cl = FilterDecimalColumnNotBetween.class;
     } else if (isDateFamily(colType) && !notKeywordPresent) {
       cl =  (hasDynamicValues ?
-          FilterLongColumnBetweenDynamicValue.class :
+          FilterDateColumnBetweenDynamicValue.class :
           FilterLongColumnBetween.class);
     } else if (isDateFamily(colType) && notKeywordPresent) {
       cl = FilterLongColumnNotBetween.class;

http://git-wip-us.apache.org/repos/asf/hive/blob/f3790ce2/ql/src/test/queries/clientpositive/vectorized_dynamic_semijoin_reduction2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/vectorized_dynamic_semijoin_reduction2.q b/ql/src/test/queries/clientpositive/vectorized_dynamic_semijoin_reduction2.q
new file mode 100644
index 0000000..446407d
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/vectorized_dynamic_semijoin_reduction2.q
@@ -0,0 +1,50 @@
+set hive.compute.query.using.stats=false;
+set hive.mapred.mode=nonstrict;
+set hive.explain.user=false;
+set hive.optimize.ppd=true;
+set hive.ppd.remove.duplicatefilters=true;
+set hive.tez.dynamic.partition.pruning=true;
+set hive.tez.dynamic.semijoin.reduction=true;
+set hive.optimize.metadataonly=false;
+set hive.optimize.index.filter=true;
+
+set hive.vectorized.adaptor.usage.mode=none;
+set hive.vectorized.execution.enabled=true;
+
+-- Create Tables
+create table dsrv2_big stored as orc as
+  select
+  cast(L_PARTKEY as bigint) as partkey_bigint,
+  cast(L_PARTKEY as decimal(10,1)) as partkey_decimal,
+  cast(L_PARTKEY as double) as partkey_double,
+  cast(l_shipdate as date) as shipdate_date,
+  cast(cast(l_shipdate as date) as timestamp) as shipdate_ts
+  from lineitem;
+create table dsrv2_small stored as orc as select * from dsrv2_big limit 20;
+analyze table dsrv2_big compute statistics;
+analyze table dsrv2_small compute statistics;
+analyze table dsrv2_big compute statistics for columns;
+analyze table dsrv2_small compute statistics for columns;
+
+-- single key (bigint)
+EXPLAIN select count(*) from dsrv2_big a join dsrv2_small b on (a.partkey_bigint = b.partkey_bigint);
+select count(*) from dsrv2_big a join dsrv2_small b on (a.partkey_bigint = b.partkey_bigint);
+
+-- single key (decimal)
+EXPLAIN select count(*) from dsrv2_big a join dsrv2_small b on (a.partkey_decimal = b.partkey_decimal);
+select count(*) from dsrv2_big a join dsrv2_small b on (a.partkey_decimal = b.partkey_decimal);
+
+-- single key (double)
+EXPLAIN select count(*) from dsrv2_big a join dsrv2_small b on (a.partkey_double = b.partkey_double);
+select count(*) from dsrv2_big a join dsrv2_small b on (a.partkey_double = b.partkey_double);
+
+-- single key (date)
+EXPLAIN select count(*) from dsrv2_big a join dsrv2_small b on (a.shipdate_date = b.shipdate_date);
+select count(*) from dsrv2_big a join dsrv2_small b on (a.shipdate_date = b.shipdate_date);
+
+-- single key (timestamp)
+EXPLAIN select count(*) from dsrv2_big a join dsrv2_small b on (a.shipdate_ts = b.shipdate_ts);
+select count(*) from dsrv2_big a join dsrv2_small b on (a.shipdate_ts = b.shipdate_ts);
+
+drop table dsrv2_big;
+drop table dsrv2_small;

http://git-wip-us.apache.org/repos/asf/hive/blob/f3790ce2/ql/src/test/results/clientpositive/llap/vectorized_dynamic_semijoin_reduction2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vectorized_dynamic_semijoin_reduction2.q.out b/ql/src/test/results/clientpositive/llap/vectorized_dynamic_semijoin_reduction2.q.out
new file mode 100644
index 0000000..27d8152
--- /dev/null
+++ b/ql/src/test/results/clientpositive/llap/vectorized_dynamic_semijoin_reduction2.q.out
@@ -0,0 +1,772 @@
+PREHOOK: query: create table dsrv2_big stored as orc as
+  select
+  cast(L_PARTKEY as bigint) as partkey_bigint,
+  cast(L_PARTKEY as decimal(10,1)) as partkey_decimal,
+  cast(L_PARTKEY as double) as partkey_double,
+  cast(l_shipdate as date) as shipdate_date,
+  cast(cast(l_shipdate as date) as timestamp) as shipdate_ts
+  from lineitem
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@lineitem
+PREHOOK: Output: database:default
+PREHOOK: Output: default@dsrv2_big
+POSTHOOK: query: create table dsrv2_big stored as orc as
+  select
+  cast(L_PARTKEY as bigint) as partkey_bigint,
+  cast(L_PARTKEY as decimal(10,1)) as partkey_decimal,
+  cast(L_PARTKEY as double) as partkey_double,
+  cast(l_shipdate as date) as shipdate_date,
+  cast(cast(l_shipdate as date) as timestamp) as shipdate_ts
+  from lineitem
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@lineitem
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@dsrv2_big
+POSTHOOK: Lineage: dsrv2_big.partkey_bigint EXPRESSION [(lineitem)lineitem.FieldSchema(name:l_partkey, type:int, comment:null), ]
+POSTHOOK: Lineage: dsrv2_big.partkey_decimal EXPRESSION [(lineitem)lineitem.FieldSchema(name:l_partkey, type:int, comment:null), ]
+POSTHOOK: Lineage: dsrv2_big.partkey_double EXPRESSION [(lineitem)lineitem.FieldSchema(name:l_partkey, type:int, comment:null), ]
+POSTHOOK: Lineage: dsrv2_big.shipdate_date EXPRESSION [(lineitem)lineitem.FieldSchema(name:l_shipdate, type:string, comment:null), ]
+POSTHOOK: Lineage: dsrv2_big.shipdate_ts EXPRESSION [(lineitem)lineitem.FieldSchema(name:l_shipdate, type:string, comment:null), ]
+PREHOOK: query: create table dsrv2_small stored as orc as select * from dsrv2_big limit 20
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@dsrv2_big
+PREHOOK: Output: database:default
+PREHOOK: Output: default@dsrv2_small
+POSTHOOK: query: create table dsrv2_small stored as orc as select * from dsrv2_big limit 20
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@dsrv2_big
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@dsrv2_small
+POSTHOOK: Lineage: dsrv2_small.partkey_bigint SIMPLE [(dsrv2_big)dsrv2_big.FieldSchema(name:partkey_bigint, type:bigint, comment:null), ]
+POSTHOOK: Lineage: dsrv2_small.partkey_decimal SIMPLE [(dsrv2_big)dsrv2_big.FieldSchema(name:partkey_decimal, type:decimal(10,1), comment:null), ]
+POSTHOOK: Lineage: dsrv2_small.partkey_double SIMPLE [(dsrv2_big)dsrv2_big.FieldSchema(name:partkey_double, type:double, comment:null), ]
+POSTHOOK: Lineage: dsrv2_small.shipdate_date SIMPLE [(dsrv2_big)dsrv2_big.FieldSchema(name:shipdate_date, type:date, comment:null), ]
+POSTHOOK: Lineage: dsrv2_small.shipdate_ts SIMPLE [(dsrv2_big)dsrv2_big.FieldSchema(name:shipdate_ts, type:timestamp, comment:null), ]
+PREHOOK: query: analyze table dsrv2_big compute statistics
+PREHOOK: type: QUERY
+PREHOOK: Input: default@dsrv2_big
+PREHOOK: Output: default@dsrv2_big
+POSTHOOK: query: analyze table dsrv2_big compute statistics
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@dsrv2_big
+POSTHOOK: Output: default@dsrv2_big
+PREHOOK: query: analyze table dsrv2_small compute statistics
+PREHOOK: type: QUERY
+PREHOOK: Input: default@dsrv2_small
+PREHOOK: Output: default@dsrv2_small
+POSTHOOK: query: analyze table dsrv2_small compute statistics
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@dsrv2_small
+POSTHOOK: Output: default@dsrv2_small
+PREHOOK: query: analyze table dsrv2_big compute statistics for columns
+PREHOOK: type: QUERY
+PREHOOK: Input: default@dsrv2_big
+#### A masked pattern was here ####
+POSTHOOK: query: analyze table dsrv2_big compute statistics for columns
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@dsrv2_big
+#### A masked pattern was here ####
+PREHOOK: query: analyze table dsrv2_small compute statistics for columns
+PREHOOK: type: QUERY
+PREHOOK: Input: default@dsrv2_small
+#### A masked pattern was here ####
+POSTHOOK: query: analyze table dsrv2_small compute statistics for columns
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@dsrv2_small
+#### A masked pattern was here ####
+PREHOOK: query: EXPLAIN select count(*) from dsrv2_big a join dsrv2_small b on (a.partkey_bigint = b.partkey_bigint)
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN select count(*) from dsrv2_big a join dsrv2_small b on (a.partkey_bigint = b.partkey_bigint)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Map 1 <- Reducer 5 (BROADCAST_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE)
+        Reducer 5 <- Map 4 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  filterExpr: (partkey_bigint is not null and partkey_bigint BETWEEN DynamicValue(RS_7_b_partkey_bigint_min) AND DynamicValue(RS_7_b_partkey_bigint_max) and in_bloom_filter(partkey_bigint, DynamicValue(RS_7_b_partkey_bigint_bloom_filter))) (type: boolean)
+                  Statistics: Num rows: 100 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: (partkey_bigint is not null and partkey_bigint BETWEEN DynamicValue(RS_7_b_partkey_bigint_min) AND DynamicValue(RS_7_b_partkey_bigint_max) and in_bloom_filter(partkey_bigint, DynamicValue(RS_7_b_partkey_bigint_bloom_filter))) (type: boolean)
+                    Statistics: Num rows: 100 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: partkey_bigint (type: bigint)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 100 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: bigint)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: bigint)
+                        Statistics: Num rows: 100 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Map 4 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  filterExpr: partkey_bigint is not null (type: boolean)
+                  Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: partkey_bigint is not null (type: boolean)
+                    Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: partkey_bigint (type: bigint)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: bigint)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: bigint)
+                        Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE
+                      Select Operator
+                        expressions: _col0 (type: bigint)
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE
+                        Group By Operator
+                          aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=18)
+                          mode: hash
+                          outputColumnNames: _col0, _col1, _col2
+                          Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                          Reduce Output Operator
+                            sort order: 
+                            Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                            value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary)
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: bigint)
+                  1 _col0 (type: bigint)
+                Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE
+                Group By Operator
+                  aggregations: count()
+                  mode: hash
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                  Reduce Output Operator
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    value expressions: _col0 (type: bigint)
+        Reducer 3 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Reducer 5 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=18)
+                mode: final
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary)
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select count(*) from dsrv2_big a join dsrv2_small b on (a.partkey_bigint = b.partkey_bigint)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@dsrv2_big
+PREHOOK: Input: default@dsrv2_small
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from dsrv2_big a join dsrv2_small b on (a.partkey_bigint = b.partkey_bigint)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@dsrv2_big
+POSTHOOK: Input: default@dsrv2_small
+#### A masked pattern was here ####
+20
+PREHOOK: query: EXPLAIN select count(*) from dsrv2_big a join dsrv2_small b on (a.partkey_decimal = b.partkey_decimal)
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN select count(*) from dsrv2_big a join dsrv2_small b on (a.partkey_decimal = b.partkey_decimal)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Map 1 <- Reducer 5 (BROADCAST_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE)
+        Reducer 5 <- Map 4 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  filterExpr: (partkey_decimal is not null and partkey_decimal BETWEEN DynamicValue(RS_7_b_partkey_decimal_min) AND DynamicValue(RS_7_b_partkey_decimal_max) and in_bloom_filter(partkey_decimal, DynamicValue(RS_7_b_partkey_decimal_bloom_filter))) (type: boolean)
+                  Statistics: Num rows: 100 Data size: 11200 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: (partkey_decimal is not null and partkey_decimal BETWEEN DynamicValue(RS_7_b_partkey_decimal_min) AND DynamicValue(RS_7_b_partkey_decimal_max) and in_bloom_filter(partkey_decimal, DynamicValue(RS_7_b_partkey_decimal_bloom_filter))) (type: boolean)
+                    Statistics: Num rows: 100 Data size: 11200 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: partkey_decimal (type: decimal(10,1))
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 100 Data size: 11200 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: decimal(10,1))
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: decimal(10,1))
+                        Statistics: Num rows: 100 Data size: 11200 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: llap
+            LLAP IO: all inputs
+        Map 4 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  filterExpr: partkey_decimal is not null (type: boolean)
+                  Statistics: Num rows: 20 Data size: 2240 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: partkey_decimal is not null (type: boolean)
+                    Statistics: Num rows: 20 Data size: 2240 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: partkey_decimal (type: decimal(10,1))
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 20 Data size: 2240 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: decimal(10,1))
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: decimal(10,1))
+                        Statistics: Num rows: 20 Data size: 2240 Basic stats: COMPLETE Column stats: COMPLETE
+                      Select Operator
+                        expressions: _col0 (type: decimal(10,1))
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 20 Data size: 2240 Basic stats: COMPLETE Column stats: COMPLETE
+                        Group By Operator
+                          aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=16)
+                          mode: hash
+                          outputColumnNames: _col0, _col1, _col2
+                          Statistics: Num rows: 1 Data size: 336 Basic stats: COMPLETE Column stats: COMPLETE
+                          Reduce Output Operator
+                            sort order: 
+                            Statistics: Num rows: 1 Data size: 336 Basic stats: COMPLETE Column stats: COMPLETE
+                            value expressions: _col0 (type: decimal(10,1)), _col1 (type: decimal(10,1)), _col2 (type: binary)
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: decimal(10,1))
+                  1 _col0 (type: decimal(10,1))
+                Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE
+                Group By Operator
+                  aggregations: count()
+                  mode: hash
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                  Reduce Output Operator
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    value expressions: _col0 (type: bigint)
+        Reducer 3 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Reducer 5 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=16)
+                mode: final
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 336 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 336 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: decimal(10,1)), _col1 (type: decimal(10,1)), _col2 (type: binary)
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select count(*) from dsrv2_big a join dsrv2_small b on (a.partkey_decimal = b.partkey_decimal)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@dsrv2_big
+PREHOOK: Input: default@dsrv2_small
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from dsrv2_big a join dsrv2_small b on (a.partkey_decimal = b.partkey_decimal)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@dsrv2_big
+POSTHOOK: Input: default@dsrv2_small
+#### A masked pattern was here ####
+20
+PREHOOK: query: EXPLAIN select count(*) from dsrv2_big a join dsrv2_small b on (a.partkey_double = b.partkey_double)
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN select count(*) from dsrv2_big a join dsrv2_small b on (a.partkey_double = b.partkey_double)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Map 1 <- Reducer 5 (BROADCAST_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE)
+        Reducer 5 <- Map 4 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  filterExpr: (partkey_double is not null and partkey_double BETWEEN DynamicValue(RS_7_b_partkey_double_min) AND DynamicValue(RS_7_b_partkey_double_max) and in_bloom_filter(partkey_double, DynamicValue(RS_7_b_partkey_double_bloom_filter))) (type: boolean)
+                  Statistics: Num rows: 100 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: (partkey_double is not null and partkey_double BETWEEN DynamicValue(RS_7_b_partkey_double_min) AND DynamicValue(RS_7_b_partkey_double_max) and in_bloom_filter(partkey_double, DynamicValue(RS_7_b_partkey_double_bloom_filter))) (type: boolean)
+                    Statistics: Num rows: 100 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: partkey_double (type: double)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 100 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: double)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: double)
+                        Statistics: Num rows: 100 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Map 4 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  filterExpr: partkey_double is not null (type: boolean)
+                  Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: partkey_double is not null (type: boolean)
+                    Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: partkey_double (type: double)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: double)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: double)
+                        Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE
+                      Select Operator
+                        expressions: _col0 (type: double)
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE
+                        Group By Operator
+                          aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=30)
+                          mode: hash
+                          outputColumnNames: _col0, _col1, _col2
+                          Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                          Reduce Output Operator
+                            sort order: 
+                            Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                            value expressions: _col0 (type: double), _col1 (type: double), _col2 (type: binary)
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: double)
+                  1 _col0 (type: double)
+                Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE
+                Group By Operator
+                  aggregations: count()
+                  mode: hash
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                  Reduce Output Operator
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    value expressions: _col0 (type: bigint)
+        Reducer 3 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Reducer 5 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=30)
+                mode: final
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: double), _col1 (type: double), _col2 (type: binary)
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select count(*) from dsrv2_big a join dsrv2_small b on (a.partkey_double = b.partkey_double)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@dsrv2_big
+PREHOOK: Input: default@dsrv2_small
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from dsrv2_big a join dsrv2_small b on (a.partkey_double = b.partkey_double)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@dsrv2_big
+POSTHOOK: Input: default@dsrv2_small
+#### A masked pattern was here ####
+20
+PREHOOK: query: EXPLAIN select count(*) from dsrv2_big a join dsrv2_small b on (a.shipdate_date = b.shipdate_date)
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN select count(*) from dsrv2_big a join dsrv2_small b on (a.shipdate_date = b.shipdate_date)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Map 1 <- Reducer 5 (BROADCAST_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE)
+        Reducer 5 <- Map 4 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  filterExpr: (shipdate_date is not null and shipdate_date BETWEEN DynamicValue(RS_7_b_shipdate_date_min) AND DynamicValue(RS_7_b_shipdate_date_max) and in_bloom_filter(shipdate_date, DynamicValue(RS_7_b_shipdate_date_bloom_filter))) (type: boolean)
+                  Statistics: Num rows: 100 Data size: 5600 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: (shipdate_date is not null and shipdate_date BETWEEN DynamicValue(RS_7_b_shipdate_date_min) AND DynamicValue(RS_7_b_shipdate_date_max) and in_bloom_filter(shipdate_date, DynamicValue(RS_7_b_shipdate_date_bloom_filter))) (type: boolean)
+                    Statistics: Num rows: 100 Data size: 5600 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: shipdate_date (type: date)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 100 Data size: 5600 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: date)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: date)
+                        Statistics: Num rows: 100 Data size: 5600 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Map 4 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  filterExpr: shipdate_date is not null (type: boolean)
+                  Statistics: Num rows: 20 Data size: 1120 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: shipdate_date is not null (type: boolean)
+                    Statistics: Num rows: 20 Data size: 1120 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: shipdate_date (type: date)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 20 Data size: 1120 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: date)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: date)
+                        Statistics: Num rows: 20 Data size: 1120 Basic stats: COMPLETE Column stats: COMPLETE
+                      Select Operator
+                        expressions: _col0 (type: date)
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 20 Data size: 1120 Basic stats: COMPLETE Column stats: COMPLETE
+                        Group By Operator
+                          aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=20)
+                          mode: hash
+                          outputColumnNames: _col0, _col1, _col2
+                          Statistics: Num rows: 1 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
+                          Reduce Output Operator
+                            sort order: 
+                            Statistics: Num rows: 1 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
+                            value expressions: _col0 (type: date), _col1 (type: date), _col2 (type: binary)
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: date)
+                  1 _col0 (type: date)
+                Statistics: Num rows: 2000 Data size: 16000 Basic stats: COMPLETE Column stats: COMPLETE
+                Group By Operator
+                  aggregations: count()
+                  mode: hash
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                  Reduce Output Operator
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    value expressions: _col0 (type: bigint)
+        Reducer 3 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Reducer 5 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=20)
+                mode: final
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: date), _col1 (type: date), _col2 (type: binary)
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select count(*) from dsrv2_big a join dsrv2_small b on (a.shipdate_date = b.shipdate_date)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@dsrv2_big
+PREHOOK: Input: default@dsrv2_small
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from dsrv2_big a join dsrv2_small b on (a.shipdate_date = b.shipdate_date)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@dsrv2_big
+POSTHOOK: Input: default@dsrv2_small
+#### A masked pattern was here ####
+23
+PREHOOK: query: EXPLAIN select count(*) from dsrv2_big a join dsrv2_small b on (a.shipdate_ts = b.shipdate_ts)
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN select count(*) from dsrv2_big a join dsrv2_small b on (a.shipdate_ts = b.shipdate_ts)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Map 1 <- Reducer 5 (BROADCAST_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE)
+        Reducer 5 <- Map 4 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  filterExpr: (shipdate_ts is not null and shipdate_ts BETWEEN DynamicValue(RS_7_b_shipdate_ts_min) AND DynamicValue(RS_7_b_shipdate_ts_max) and in_bloom_filter(shipdate_ts, DynamicValue(RS_7_b_shipdate_ts_bloom_filter))) (type: boolean)
+                  Statistics: Num rows: 100 Data size: 4000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: (shipdate_ts is not null and shipdate_ts BETWEEN DynamicValue(RS_7_b_shipdate_ts_min) AND DynamicValue(RS_7_b_shipdate_ts_max) and in_bloom_filter(shipdate_ts, DynamicValue(RS_7_b_shipdate_ts_bloom_filter))) (type: boolean)
+                    Statistics: Num rows: 100 Data size: 4000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: shipdate_ts (type: timestamp)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 100 Data size: 4000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: timestamp)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: timestamp)
+                        Statistics: Num rows: 100 Data size: 4000 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: llap
+            LLAP IO: all inputs
+        Map 4 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  filterExpr: shipdate_ts is not null (type: boolean)
+                  Statistics: Num rows: 20 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: shipdate_ts is not null (type: boolean)
+                    Statistics: Num rows: 20 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: shipdate_ts (type: timestamp)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 20 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: timestamp)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: timestamp)
+                        Statistics: Num rows: 20 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
+                      Select Operator
+                        expressions: _col0 (type: timestamp)
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 20 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
+                        Group By Operator
+                          aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=20)
+                          mode: hash
+                          outputColumnNames: _col0, _col1, _col2
+                          Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE
+                          Reduce Output Operator
+                            sort order: 
+                            Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE
+                            value expressions: _col0 (type: timestamp), _col1 (type: timestamp), _col2 (type: binary)
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: timestamp)
+                  1 _col0 (type: timestamp)
+                Statistics: Num rows: 2000 Data size: 16000 Basic stats: COMPLETE Column stats: COMPLETE
+                Group By Operator
+                  aggregations: count()
+                  mode: hash
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                  Reduce Output Operator
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    value expressions: _col0 (type: bigint)
+        Reducer 3 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Reducer 5 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=20)
+                mode: final
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: timestamp), _col1 (type: timestamp), _col2 (type: binary)
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select count(*) from dsrv2_big a join dsrv2_small b on (a.shipdate_ts = b.shipdate_ts)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@dsrv2_big
+PREHOOK: Input: default@dsrv2_small
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from dsrv2_big a join dsrv2_small b on (a.shipdate_ts = b.shipdate_ts)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@dsrv2_big
+POSTHOOK: Input: default@dsrv2_small
+#### A masked pattern was here ####
+23
+PREHOOK: query: drop table dsrv2_big
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@dsrv2_big
+PREHOOK: Output: default@dsrv2_big
+POSTHOOK: query: drop table dsrv2_big
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@dsrv2_big
+POSTHOOK: Output: default@dsrv2_big
+PREHOOK: query: drop table dsrv2_small
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@dsrv2_small
+PREHOOK: Output: default@dsrv2_small
+POSTHOOK: query: drop table dsrv2_small
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@dsrv2_small
+POSTHOOK: Output: default@dsrv2_small

http://git-wip-us.apache.org/repos/asf/hive/blob/f3790ce2/vector-code-gen/src/org/apache/hadoop/hive/tools/GenVectorCode.java
----------------------------------------------------------------------
diff --git a/vector-code-gen/src/org/apache/hadoop/hive/tools/GenVectorCode.java b/vector-code-gen/src/org/apache/hadoop/hive/tools/GenVectorCode.java
index 22b8752..55cfb7b 100644
--- a/vector-code-gen/src/org/apache/hadoop/hive/tools/GenVectorCode.java
+++ b/vector-code-gen/src/org/apache/hadoop/hive/tools/GenVectorCode.java
@@ -796,6 +796,7 @@ public class GenVectorCode extends Task {
       {"FilterColumnBetweenDynamicValue", "string", ""},
       {"FilterColumnBetweenDynamicValue", "char", ""},
       {"FilterColumnBetweenDynamicValue", "varchar", ""},
+      {"FilterColumnBetweenDynamicValue", "date", ""},
       {"FilterColumnBetweenDynamicValue", "timestamp", ""},
 
       {"ColumnCompareColumn", "Equal", "long", "double", "=="},
@@ -1402,42 +1403,58 @@ public class GenVectorCode extends Task {
     String vectorType;
     String getPrimitiveMethod;
     String getValueMethod;
+    String conversionMethod;
 
     if (operandType.equals("long")) {
       defaultValue = "0";
       vectorType = "long";
       getPrimitiveMethod = "getLong";
       getValueMethod = "";
+      conversionMethod = "";
     } else if (operandType.equals("double")) {
       defaultValue = "0";
       vectorType = "double";
       getPrimitiveMethod = "getDouble";
       getValueMethod = "";
+      conversionMethod = "";
     } else if (operandType.equals("decimal")) {
       defaultValue = "null";
       vectorType = "HiveDecimal";
       getPrimitiveMethod = "getHiveDecimal";
       getValueMethod = "";
+      conversionMethod = "";
     } else if (operandType.equals("string")) {
       defaultValue = "null";
       vectorType = "byte[]";
       getPrimitiveMethod = "getString";
       getValueMethod = ".getBytes()";
+      conversionMethod = "";
     } else if (operandType.equals("char")) {
       defaultValue = "null";
       vectorType = "byte[]";
       getPrimitiveMethod = "getHiveChar";
       getValueMethod = ".getStrippedValue().getBytes()";  // Does vectorization use stripped char values?
+      conversionMethod = "";
     } else if (operandType.equals("varchar")) {
       defaultValue = "null";
       vectorType = "byte[]";
       getPrimitiveMethod = "getHiveVarchar";
       getValueMethod = ".getValue().getBytes()";
+      conversionMethod = "";
+    } else if (operandType.equals("date")) {
+      defaultValue = "0";
+      vectorType = "long";
+      getPrimitiveMethod = "getDate";
+      getValueMethod = "";
+      conversionMethod = "DateWritable.dateToDays";
+      // Special case - Date requires its own specific BetweenDynamicValue class, but derives from FilterLongColumnBetween
+      typeName = "Long";
     } else if (operandType.equals("timestamp")) {
       defaultValue = "null";
       vectorType = "Timestamp";
       getPrimitiveMethod = "getTimestamp";
       getValueMethod = "";
+      conversionMethod = "";
     } else {
       throw new IllegalArgumentException("Type " + operandType + " not supported");
     }
@@ -1451,6 +1468,7 @@ public class GenVectorCode extends Task {
     templateString = templateString.replaceAll("<VectorType>", vectorType);
     templateString = templateString.replaceAll("<GetPrimitiveMethod>", getPrimitiveMethod);
     templateString = templateString.replaceAll("<GetValueMethod>", getValueMethod);
+    templateString = templateString.replaceAll("<ConversionMethod>", conversionMethod);
 
     writeFile(templateFile.lastModified(), expressionOutputDirectory, expressionClassesDirectory,
         className, templateString);