You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by mm...@apache.org on 2016/05/28 03:30:02 UTC

[1/5] hive git commit: HIVE-13084: Vectorization add support for PROJECTION Multi-AND/OR (Matt McCline, reviewed by Sergey Shelukhin)

Repository: hive
Updated Branches:
  refs/heads/branch-2.1 2bd2e9a09 -> 9dd70f0d3


http://git-wip-us.apache.org/repos/asf/hive/blob/9dd70f0d/ql/src/test/results/clientpositive/vector_multi_or_projection.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_multi_or_projection.q.out b/ql/src/test/results/clientpositive/vector_multi_or_projection.q.out
new file mode 100644
index 0000000..17f2d8b
--- /dev/null
+++ b/ql/src/test/results/clientpositive/vector_multi_or_projection.q.out
@@ -0,0 +1,800 @@
+PREHOOK: query: -- SORT_QUERY_RESULTS
+
+create table vectortab2k(
+            t tinyint,
+            si smallint,
+            i int,
+            b bigint,
+            f float,
+            d double,
+            dc decimal(38,18),
+            bo boolean,
+            s string,
+            s2 string,
+            ts timestamp,
+            ts2 timestamp,
+            dt date)
+ROW FORMAT DELIMITED FIELDS TERMINATED BY '|'
+STORED AS TEXTFILE
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@vectortab2k
+POSTHOOK: query: -- SORT_QUERY_RESULTS
+
+create table vectortab2k(
+            t tinyint,
+            si smallint,
+            i int,
+            b bigint,
+            f float,
+            d double,
+            dc decimal(38,18),
+            bo boolean,
+            s string,
+            s2 string,
+            ts timestamp,
+            ts2 timestamp,
+            dt date)
+ROW FORMAT DELIMITED FIELDS TERMINATED BY '|'
+STORED AS TEXTFILE
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@vectortab2k
+PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/vectortab2k' OVERWRITE INTO TABLE vectortab2k
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: default@vectortab2k
+POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/vectortab2k' OVERWRITE INTO TABLE vectortab2k
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: default@vectortab2k
+PREHOOK: query: CREATE TABLE scratch AS SELECT t, si, i, b, bo FROM vectortab2k
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@vectortab2k
+PREHOOK: Output: database:default
+PREHOOK: Output: default@scratch
+POSTHOOK: query: CREATE TABLE scratch AS SELECT t, si, i, b, bo FROM vectortab2k
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@vectortab2k
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@scratch
+POSTHOOK: Lineage: scratch.b SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: scratch.bo SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: scratch.i SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: scratch.si SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: scratch.t SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:t, type:tinyint, comment:null), ]
+t	si	i	b	bo
+PREHOOK: query: INSERT INTO TABLE scratch VALUES (NULL, NULL, NULL, NULL, NULL)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__1
+PREHOOK: Output: default@scratch
+POSTHOOK: query: INSERT INTO TABLE scratch VALUES (NULL, NULL, NULL, NULL, NULL)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__1
+POSTHOOK: Output: default@scratch
+POSTHOOK: Lineage: scratch.b EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col4, type:string, comment:), ]
+POSTHOOK: Lineage: scratch.bo EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col5, type:string, comment:), ]
+POSTHOOK: Lineage: scratch.i EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col3, type:string, comment:), ]
+POSTHOOK: Lineage: scratch.si EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+POSTHOOK: Lineage: scratch.t EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+_col0	_col1	_col2	_col3	_col4
+PREHOOK: query: CREATE TABLE vectortab2k_orc STORED AS ORC AS SELECT * FROM scratch
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@scratch
+PREHOOK: Output: database:default
+PREHOOK: Output: default@vectortab2k_orc
+POSTHOOK: query: CREATE TABLE vectortab2k_orc STORED AS ORC AS SELECT * FROM scratch
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@scratch
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@vectortab2k_orc
+POSTHOOK: Lineage: vectortab2k_orc.b SIMPLE [(scratch)scratch.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc.bo SIMPLE [(scratch)scratch.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc.i SIMPLE [(scratch)scratch.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc.si SIMPLE [(scratch)scratch.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc.t SIMPLE [(scratch)scratch.FieldSchema(name:t, type:tinyint, comment:null), ]
+scratch.t	scratch.si	scratch.i	scratch.b	scratch.bo
+PREHOOK: query: EXPLAIN
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t < 0 OR si > 0 OR i < 0) as multi_or_col from vectortab2k_orc
+        order by t, si, i) as q
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t < 0 OR si > 0 OR i < 0) as multi_or_col from vectortab2k_orc
+        order by t, si, i) as q
+POSTHOOK: type: QUERY
+Explain
+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: vectortab2k_orc
+            Statistics: Num rows: 2001 Data size: 45620 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: t (type: tinyint), si (type: smallint), i (type: int), (t < 0) (type: boolean), (si > 0) (type: boolean), (i < 0) (type: boolean), ((t < 0) or (si > 0) or (i < 0)) (type: boolean)
+              outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
+              Statistics: Num rows: 2001 Data size: 45620 Basic stats: COMPLETE Column stats: NONE
+              Reduce Output Operator
+                key expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: int)
+                sort order: +++
+                Statistics: Num rows: 2001 Data size: 45620 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col3 (type: boolean), _col4 (type: boolean), _col5 (type: boolean), _col6 (type: boolean)
+      Execution mode: vectorized
+      Reduce Operator Tree:
+        Select Operator
+          expressions: hash(KEY.reducesinkkey0,KEY.reducesinkkey1,KEY.reducesinkkey2,VALUE._col0,VALUE._col1,VALUE._col2,VALUE._col3) (type: int)
+          outputColumnNames: _col0
+          Statistics: Num rows: 2001 Data size: 45620 Basic stats: COMPLETE Column stats: NONE
+          Group By Operator
+            aggregations: sum(_col0)
+            mode: complete
+            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.SequenceFileInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t < 0 OR si > 0 OR i < 0) as multi_or_col from vectortab2k_orc
+        order by t, si, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t < 0 OR si > 0 OR i < 0) as multi_or_col from vectortab2k_orc
+        order by t, si, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc
+#### A masked pattern was here ####
+c0
+-64411364734
+PREHOOK: query: EXPLAIN
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, b, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 OR si > 0 OR i < 0 OR b > 0) as multi_or_col from vectortab2k_orc
+        order by t, si, i, b) as q
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, b, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 OR si > 0 OR i < 0 OR b > 0) as multi_or_col from vectortab2k_orc
+        order by t, si, i, b) as q
+POSTHOOK: type: QUERY
+Explain
+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: vectortab2k_orc
+            Statistics: Num rows: 2001 Data size: 45620 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: t (type: tinyint), si (type: smallint), i (type: int), b (type: bigint), (t < 0) (type: boolean), (si > 0) (type: boolean), (i < 0) (type: boolean), (b > 0) (type: boolean), ((t < 0) or (si > 0) or (i < 0) or (b > 0)) (type: boolean)
+              outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
+              Statistics: Num rows: 2001 Data size: 45620 Basic stats: COMPLETE Column stats: NONE
+              Reduce Output Operator
+                key expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint)
+                sort order: ++++
+                Statistics: Num rows: 2001 Data size: 45620 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col4 (type: boolean), _col5 (type: boolean), _col6 (type: boolean), _col7 (type: boolean), _col8 (type: boolean)
+      Execution mode: vectorized
+      Reduce Operator Tree:
+        Select Operator
+          expressions: hash(KEY.reducesinkkey0,KEY.reducesinkkey1,KEY.reducesinkkey2,KEY.reducesinkkey3,VALUE._col0,VALUE._col1,VALUE._col2,VALUE._col3,VALUE._col4) (type: int)
+          outputColumnNames: _col0
+          Statistics: Num rows: 2001 Data size: 45620 Basic stats: COMPLETE Column stats: NONE
+          Group By Operator
+            aggregations: sum(_col0)
+            mode: complete
+            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.SequenceFileInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, b, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 OR si > 0 OR i < 0 OR b > 0) as multi_or_col from vectortab2k_orc
+        order by t, si, i, b) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, b, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 OR si > 0 OR i < 0 OR b > 0) as multi_or_col from vectortab2k_orc
+        order by t, si, i, b) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc
+#### A masked pattern was here ####
+c0
+96890479010
+PREHOOK: query: -- Use a boolean column rather than a column comparison expression.
+EXPLAIN
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo, b, (t < 0) as child1, (si > 0) as child2, bo as child3, (b > 0) as child4, (t < 0 OR si > 0 OR bo OR b > 0) as multi_or_col from vectortab2k_orc
+        order by t, si, bo, b) as q
+PREHOOK: type: QUERY
+POSTHOOK: query: -- Use a boolean column rather than a column comparison expression.
+EXPLAIN
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo, b, (t < 0) as child1, (si > 0) as child2, bo as child3, (b > 0) as child4, (t < 0 OR si > 0 OR bo OR b > 0) as multi_or_col from vectortab2k_orc
+        order by t, si, bo, b) as q
+POSTHOOK: type: QUERY
+Explain
+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: vectortab2k_orc
+            Statistics: Num rows: 2001 Data size: 45620 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: t (type: tinyint), si (type: smallint), bo (type: boolean), b (type: bigint), (t < 0) (type: boolean), (si > 0) (type: boolean), (b > 0) (type: boolean), ((t < 0) or (si > 0) or bo or (b > 0)) (type: boolean)
+              outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col7, _col8
+              Statistics: Num rows: 2001 Data size: 45620 Basic stats: COMPLETE Column stats: NONE
+              Reduce Output Operator
+                key expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: boolean), _col3 (type: bigint)
+                sort order: ++++
+                Statistics: Num rows: 2001 Data size: 45620 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col4 (type: boolean), _col5 (type: boolean), _col7 (type: boolean), _col8 (type: boolean)
+      Execution mode: vectorized
+      Reduce Operator Tree:
+        Select Operator
+          expressions: hash(KEY.reducesinkkey0,KEY.reducesinkkey1,KEY.reducesinkkey2,KEY.reducesinkkey3,VALUE._col0,VALUE._col1,KEY.reducesinkkey2,VALUE._col2,VALUE._col3) (type: int)
+          outputColumnNames: _col0
+          Statistics: Num rows: 2001 Data size: 45620 Basic stats: COMPLETE Column stats: NONE
+          Group By Operator
+            aggregations: sum(_col0)
+            mode: complete
+            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.SequenceFileInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo, b, (t < 0) as child1, (si > 0) as child2, bo as child3, (b > 0) as child4, (t < 0 OR si > 0 OR bo OR b > 0) as multi_or_col from vectortab2k_orc
+        order by t, si, bo, b) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo, b, (t < 0) as child1, (si > 0) as child2, bo as child3, (b > 0) as child4, (t < 0 OR si > 0 OR bo OR b > 0) as multi_or_col from vectortab2k_orc
+        order by t, si, bo, b) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc
+#### A masked pattern was here ####
+c0
+26141426137
+PREHOOK: query: -- With some filtering
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t < 0 OR si > 0 OR i < 0) as multi_or_col from vectortab2k_orc
+        where pmod(i,4) = 2
+        order by t, si, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc
+#### A masked pattern was here ####
+POSTHOOK: query: -- With some filtering
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t < 0 OR si > 0 OR i < 0) as multi_or_col from vectortab2k_orc
+        where pmod(i,4) = 2
+        order by t, si, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc
+#### A masked pattern was here ####
+c0
+-36533047758
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, b, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 OR si > 0 OR i < 0 OR b > 0) as multi_or_col from vectortab2k_orc
+        where pmod(si,4) < 2
+        order by t, si, i, b) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, b, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 OR si > 0 OR i < 0 OR b > 0) as multi_or_col from vectortab2k_orc
+        where pmod(si,4) < 2
+        order by t, si, i, b) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc
+#### A masked pattern was here ####
+c0
+41220563314
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo, b, (t < 0) as child1, (si > 0) as child2, bo as child3, (b > 0) as child4, (t < 0 OR si > 0 OR bo OR b > 0) as multi_or_col from vectortab2k_orc
+        where pmod(i,4) = 2
+        order by t, si, bo, b) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo, b, (t < 0) as child1, (si > 0) as child2, bo as child3, (b > 0) as child4, (t < 0 OR si > 0 OR bo OR b > 0) as multi_or_col from vectortab2k_orc
+        where pmod(i,4) = 2
+        order by t, si, bo, b) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc
+#### A masked pattern was here ####
+c0
+13203204057
+PREHOOK: query: CREATE TABLE scratch_repeat AS SELECT t, si, i, b, bo,
+    20 as t_repeat, 9000 as si_repeat, 9233320 as i_repeat, -823823999339992 as b_repeat, false as bo_repeat_false, true as bo_repeat_true FROM vectortab2k
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@vectortab2k
+PREHOOK: Output: database:default
+PREHOOK: Output: default@scratch_repeat
+POSTHOOK: query: CREATE TABLE scratch_repeat AS SELECT t, si, i, b, bo,
+    20 as t_repeat, 9000 as si_repeat, 9233320 as i_repeat, -823823999339992 as b_repeat, false as bo_repeat_false, true as bo_repeat_true FROM vectortab2k
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@vectortab2k
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@scratch_repeat
+POSTHOOK: Lineage: scratch_repeat.b SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: scratch_repeat.b_repeat SIMPLE []
+POSTHOOK: Lineage: scratch_repeat.bo SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: scratch_repeat.bo_repeat_false SIMPLE []
+POSTHOOK: Lineage: scratch_repeat.bo_repeat_true SIMPLE []
+POSTHOOK: Lineage: scratch_repeat.i SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: scratch_repeat.i_repeat SIMPLE []
+POSTHOOK: Lineage: scratch_repeat.si SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: scratch_repeat.si_repeat SIMPLE []
+POSTHOOK: Lineage: scratch_repeat.t SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:t, type:tinyint, comment:null), ]
+POSTHOOK: Lineage: scratch_repeat.t_repeat SIMPLE []
+t	si	i	b	bo	t_repeat	si_repeat	i_repeat	b_repeat	bo_repeat_false	bo_repeat_true
+PREHOOK: query: -- The repeated columns ought to create repeated VectorizedRowBatch for those columns.
+-- And then when we do a comparison, we should generate a repeated boolean result.
+CREATE TABLE vectortab2k_orc_repeat STORED AS ORC AS SELECT * FROM scratch_repeat
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@scratch_repeat
+PREHOOK: Output: database:default
+PREHOOK: Output: default@vectortab2k_orc_repeat
+POSTHOOK: query: -- The repeated columns ought to create repeated VectorizedRowBatch for those columns.
+-- And then when we do a comparison, we should generate a repeated boolean result.
+CREATE TABLE vectortab2k_orc_repeat STORED AS ORC AS SELECT * FROM scratch_repeat
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@scratch_repeat
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@vectortab2k_orc_repeat
+POSTHOOK: Lineage: vectortab2k_orc_repeat.b SIMPLE [(scratch_repeat)scratch_repeat.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_repeat.b_repeat SIMPLE [(scratch_repeat)scratch_repeat.FieldSchema(name:b_repeat, type:bigint, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_repeat.bo SIMPLE [(scratch_repeat)scratch_repeat.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_repeat.bo_repeat_false SIMPLE [(scratch_repeat)scratch_repeat.FieldSchema(name:bo_repeat_false, type:boolean, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_repeat.bo_repeat_true SIMPLE [(scratch_repeat)scratch_repeat.FieldSchema(name:bo_repeat_true, type:boolean, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_repeat.i SIMPLE [(scratch_repeat)scratch_repeat.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_repeat.i_repeat SIMPLE [(scratch_repeat)scratch_repeat.FieldSchema(name:i_repeat, type:int, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_repeat.si SIMPLE [(scratch_repeat)scratch_repeat.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_repeat.si_repeat SIMPLE [(scratch_repeat)scratch_repeat.FieldSchema(name:si_repeat, type:int, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_repeat.t SIMPLE [(scratch_repeat)scratch_repeat.FieldSchema(name:t, type:tinyint, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_repeat.t_repeat SIMPLE [(scratch_repeat)scratch_repeat.FieldSchema(name:t_repeat, type:int, comment:null), ]
+scratch_repeat.t	scratch_repeat.si	scratch_repeat.i	scratch_repeat.b	scratch_repeat.bo	scratch_repeat.t_repeat	scratch_repeat.si_repeat	scratch_repeat.i_repeat	scratch_repeat.b_repeat	scratch_repeat.bo_repeat_false	scratch_repeat.bo_repeat_true
+PREHOOK: query: -- t_repeat < 0 should generate all false.
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, i, (t_repeat < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_repeat < 0 OR si > 0 OR i < 0) as multi_or_col from vectortab2k_orc_repeat
+        order by t_repeat, si, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: -- t_repeat < 0 should generate all false.
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, i, (t_repeat < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_repeat < 0 OR si > 0 OR i < 0) as multi_or_col from vectortab2k_orc_repeat
+        order by t_repeat, si, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+-53707877629
+PREHOOK: query: -- t_repeat > 0 should generate all true.
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, i, (t_repeat > 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_repeat > 0 OR si > 0 OR i < 0) as multi_or_col from vectortab2k_orc_repeat
+        order by t_repeat, si, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: -- t_repeat > 0 should generate all true.
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, i, (t_repeat > 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_repeat > 0 OR si > 0 OR i < 0) as multi_or_col from vectortab2k_orc_repeat
+        order by t_repeat, si, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+-53648295050
+PREHOOK: query: -- Two repeated false columns at beginning...
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si_repeat, i, (t_repeat < 0) as child1, (si_repeat < 0) as child2, (i < 0) as child3, (t_repeat < 0 OR si_repeat < 0 OR i < 0) as multi_or_col from vectortab2k_orc_repeat
+        order by t_repeat, si_repeat, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: -- Two repeated false columns at beginning...
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si_repeat, i, (t_repeat < 0) as child1, (si_repeat < 0) as child2, (i < 0) as child3, (t_repeat < 0 OR si_repeat < 0 OR i < 0) as multi_or_col from vectortab2k_orc_repeat
+        order by t_repeat, si_repeat, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+-55182904269
+PREHOOK: query: -- si_repeat > 0 should generate all true.
+SELECT sum(hash(*)) FROM
+    (SELECT t, si_repeat, i, b, (t < 0) as child1, (si_repeat > 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 OR si_repeat > 0 OR i < 0 OR b > 0) as multi_or_col from vectortab2k_orc_repeat
+        order by t, si_repeat, i, b) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: -- si_repeat > 0 should generate all true.
+SELECT sum(hash(*)) FROM
+    (SELECT t, si_repeat, i, b, (t < 0) as child1, (si_repeat > 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 OR si_repeat > 0 OR i < 0 OR b > 0) as multi_or_col from vectortab2k_orc_repeat
+        order by t, si_repeat, i, b) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+33185964172
+PREHOOK: query: -- si_repeat < 0 should generate all false.
+SELECT sum(hash(*)) FROM
+    (SELECT t, si_repeat, i, b, (t < 0) as child1, (si_repeat < 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 OR si_repeat < 0 OR i < 0 OR b > 0) as multi_or_col from vectortab2k_orc_repeat
+        order by t, si_repeat, i, b) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: -- si_repeat < 0 should generate all false.
+SELECT sum(hash(*)) FROM
+    (SELECT t, si_repeat, i, b, (t < 0) as child1, (si_repeat < 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 OR si_repeat < 0 OR i < 0 OR b > 0) as multi_or_col from vectortab2k_orc_repeat
+        order by t, si_repeat, i, b) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+33126382002
+PREHOOK: query: -- Use a boolean column rather than a column comparison expression.
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, bo_repeat_false, b, (t_repeat > 0) as child1, (si > 0) as child2, (bo_repeat_false) as child3, (b > 0) as child4, (t_repeat > 0 OR si > 0 OR bo_repeat_false OR b > 0) as multi_or_col from vectortab2k_orc_repeat
+        order by t_repeat, si, bo_repeat_false, b) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: -- Use a boolean column rather than a column comparison expression.
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, bo_repeat_false, b, (t_repeat > 0) as child1, (si > 0) as child2, (bo_repeat_false) as child3, (b > 0) as child4, (t_repeat > 0 OR si > 0 OR bo_repeat_false OR b > 0) as multi_or_col from vectortab2k_orc_repeat
+        order by t_repeat, si, bo_repeat_false, b) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+-35841267581
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, bo_repeat_true, b, (t_repeat > 0) as child1, (si > 0) as child2, (bo_repeat_true) as child3, (b > 0) as child4, (t_repeat > 0 OR si > 0 OR bo_repeat_true OR b > 0) as multi_or_col from vectortab2k_orc_repeat
+        order by t_repeat, si, bo_repeat_true, b) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, bo_repeat_true, b, (t_repeat > 0) as child1, (si > 0) as child2, (bo_repeat_true) as child3, (b > 0) as child4, (t_repeat > 0 OR si > 0 OR bo_repeat_true OR b > 0) as multi_or_col from vectortab2k_orc_repeat
+        order by t_repeat, si, bo_repeat_true, b) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+85605607459
+PREHOOK: query: -- With some filtering
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, i, (t_repeat < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_repeat < 0 OR si > 0 OR i < 0) as multi_or_col from vectortab2k_orc_repeat
+        where pmod(i,4) > 1
+        order by t_repeat, si, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: -- With some filtering
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, i, (t_repeat < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_repeat < 0 OR si > 0 OR i < 0) as multi_or_col from vectortab2k_orc_repeat
+        where pmod(i,4) > 1
+        order by t_repeat, si, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+-30735538712
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, i, (t_repeat > 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_repeat > 0 OR si > 0 OR i < 0) as multi_or_col from vectortab2k_orc_repeat
+        where pmod(si,4) > 1
+        order by t_repeat, si, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, i, (t_repeat > 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_repeat > 0 OR si > 0 OR i < 0) as multi_or_col from vectortab2k_orc_repeat
+        where pmod(si,4) > 1
+        order by t_repeat, si, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+-16221803790
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si_repeat, i, (t_repeat < 0) as child1, (si_repeat < 0) as child2, (i < 0) as child3, (t_repeat < 0 OR si_repeat < 0 OR i < 0) as multi_or_col from vectortab2k_orc_repeat
+        where pmod(i,4) > 1
+        order by t_repeat, si_repeat, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si_repeat, i, (t_repeat < 0) as child1, (si_repeat < 0) as child2, (i < 0) as child3, (t_repeat < 0 OR si_repeat < 0 OR i < 0) as multi_or_col from vectortab2k_orc_repeat
+        where pmod(i,4) > 1
+        order by t_repeat, si_repeat, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+-56285878286
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si_repeat, i, b, (t < 0) as child1, (si_repeat > 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 OR si_repeat > 0 OR i < 0 OR b > 0) as multi_or_col from vectortab2k_orc_repeat
+        where pmod(t,4) > 1
+        order by t, si_repeat, i, b) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si_repeat, i, b, (t < 0) as child1, (si_repeat > 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 OR si_repeat > 0 OR i < 0 OR b > 0) as multi_or_col from vectortab2k_orc_repeat
+        where pmod(t,4) > 1
+        order by t, si_repeat, i, b) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+-7888615123
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si_repeat, i, b, (t < 0) as child1, (si_repeat < 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 OR si_repeat < 0 OR i < 0 OR b > 0) as multi_or_col from vectortab2k_orc_repeat
+        where pmod(b,4) > 1
+        order by t, si_repeat, i, b) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si_repeat, i, b, (t < 0) as child1, (si_repeat < 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 OR si_repeat < 0 OR i < 0 OR b > 0) as multi_or_col from vectortab2k_orc_repeat
+        where pmod(b,4) > 1
+        order by t, si_repeat, i, b) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+16350664785
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, bo_repeat_false, b, (t_repeat > 0) as child1, (si > 0) as child2, (bo_repeat_false) as child3, (b > 0) as child4, (t_repeat > 0 OR si > 0 OR bo_repeat_false OR b > 0) as multi_or_col from vectortab2k_orc_repeat
+        where pmod(si,4) > 1
+        order by t_repeat, si, bo_repeat_false, b) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, bo_repeat_false, b, (t_repeat > 0) as child1, (si > 0) as child2, (bo_repeat_false) as child3, (b > 0) as child4, (t_repeat > 0 OR si > 0 OR bo_repeat_false OR b > 0) as multi_or_col from vectortab2k_orc_repeat
+        where pmod(si,4) > 1
+        order by t_repeat, si, bo_repeat_false, b) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+-5022644323
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, bo_repeat_true, b, (t_repeat > 0) as child1, (si > 0) as child2, (bo_repeat_true) as child3, (b > 0) as child4, (t_repeat > 0 OR si > 0 OR bo_repeat_true OR b > 0) as multi_or_col from vectortab2k_orc_repeat
+        where pmod(si,4) < 2
+        order by t_repeat, si, bo_repeat_true, b) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, bo_repeat_true, b, (t_repeat > 0) as child1, (si > 0) as child2, (bo_repeat_true) as child3, (b > 0) as child4, (t_repeat > 0 OR si > 0 OR bo_repeat_true OR b > 0) as multi_or_col from vectortab2k_orc_repeat
+        where pmod(si,4) < 2
+        order by t_repeat, si, bo_repeat_true, b) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+51133657254
+PREHOOK: query: CREATE TABLE scratch_null AS SELECT t, si, i, b, bo,
+     cast(null as tinyint) as t_null, cast(null as smallint) as si_null, cast(null as int) as i_null, cast(null as bigint) as b_null, cast(null as boolean) as bo_null FROM vectortab2k
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@vectortab2k
+PREHOOK: Output: database:default
+PREHOOK: Output: default@scratch_null
+POSTHOOK: query: CREATE TABLE scratch_null AS SELECT t, si, i, b, bo,
+     cast(null as tinyint) as t_null, cast(null as smallint) as si_null, cast(null as int) as i_null, cast(null as bigint) as b_null, cast(null as boolean) as bo_null FROM vectortab2k
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@vectortab2k
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@scratch_null
+POSTHOOK: Lineage: scratch_null.b SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: scratch_null.b_null SIMPLE []
+POSTHOOK: Lineage: scratch_null.bo SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: scratch_null.bo_null SIMPLE []
+POSTHOOK: Lineage: scratch_null.i SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: scratch_null.i_null SIMPLE []
+POSTHOOK: Lineage: scratch_null.si SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: scratch_null.si_null SIMPLE []
+POSTHOOK: Lineage: scratch_null.t SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:t, type:tinyint, comment:null), ]
+POSTHOOK: Lineage: scratch_null.t_null SIMPLE []
+t	si	i	b	bo	t_null	si_null	i_null	b_null	bo_null
+PREHOOK: query: -- The nulled columns ought to create repeated null VectorizedRowBatch for those columns.
+CREATE TABLE vectortab2k_orc_null STORED AS ORC AS SELECT * FROM scratch_null
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@scratch_null
+PREHOOK: Output: database:default
+PREHOOK: Output: default@vectortab2k_orc_null
+POSTHOOK: query: -- The nulled columns ought to create repeated null VectorizedRowBatch for those columns.
+CREATE TABLE vectortab2k_orc_null STORED AS ORC AS SELECT * FROM scratch_null
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@scratch_null
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@vectortab2k_orc_null
+POSTHOOK: Lineage: vectortab2k_orc_null.b SIMPLE [(scratch_null)scratch_null.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_null.b_null SIMPLE [(scratch_null)scratch_null.FieldSchema(name:b_null, type:bigint, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_null.bo SIMPLE [(scratch_null)scratch_null.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_null.bo_null SIMPLE [(scratch_null)scratch_null.FieldSchema(name:bo_null, type:boolean, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_null.i SIMPLE [(scratch_null)scratch_null.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_null.i_null SIMPLE [(scratch_null)scratch_null.FieldSchema(name:i_null, type:int, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_null.si SIMPLE [(scratch_null)scratch_null.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_null.si_null SIMPLE [(scratch_null)scratch_null.FieldSchema(name:si_null, type:smallint, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_null.t SIMPLE [(scratch_null)scratch_null.FieldSchema(name:t, type:tinyint, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_null.t_null SIMPLE [(scratch_null)scratch_null.FieldSchema(name:t_null, type:tinyint, comment:null), ]
+scratch_null.t	scratch_null.si	scratch_null.i	scratch_null.b	scratch_null.bo	scratch_null.t_null	scratch_null.si_null	scratch_null.i_null	scratch_null.b_null	scratch_null.bo_null
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_null, si, i, (t_null is null) as child1, (si is null) as child2, (i < 0) as child3, (t_null is null OR si is null OR i < 0) as multi_and_col from vectortab2k_orc_null
+        order by t_null, si, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_null, si, i, (t_null is null) as child1, (si is null) as child2, (i < 0) as child3, (t_null is null OR si is null OR i < 0) as multi_and_col from vectortab2k_orc_null
+        order by t_null, si, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+c0
+-77366438890
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_null, si, i, (t_null < 0) as child1, (si > 0) as child2, (i is null) as child3, (t_null < 0 OR si > 0 OR i is null) as multi_and_col from vectortab2k_orc_null
+        order by t_null, si, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_null, si, i, (t_null < 0) as child1, (si > 0) as child2, (i is null) as child3, (t_null < 0 OR si > 0 OR i is null) as multi_and_col from vectortab2k_orc_null
+        order by t_null, si, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+c0
+-77425279225
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_null, si, i_null, (t_null < 0) as child1, (si > 0) as child2, (i_null < 0) as child3, (t_null < 0 OR si > 0 OR i_null < 0) as multi_and_col from vectortab2k_orc_null
+        order by t_null, si, i_null) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_null, si, i_null, (t_null < 0) as child1, (si > 0) as child2, (i_null < 0) as child3, (t_null < 0 OR si > 0 OR i_null < 0) as multi_and_col from vectortab2k_orc_null
+        order by t_null, si, i_null) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+c0
+33196722176
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si_null, i_null, (t < 0) as child1, (si_null > 0) as child2, (i_null < 0) as child3, (t < 0 OR si_null > 0 OR i_null < 0) as multi_and_col from vectortab2k_orc_null
+        order by t, si_null, i_null) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si_null, i_null, (t < 0) as child1, (si_null > 0) as child2, (i_null < 0) as child3, (t < 0 OR si_null > 0 OR i_null < 0) as multi_and_col from vectortab2k_orc_null
+        order by t, si_null, i_null) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+c0
+25898790496
+PREHOOK: query: -- With some filtering
+SELECT sum(hash(*)) FROM
+    (SELECT t_null, si, i, (t_null is null) as child1, (si is null) as child2, (i < 0) as child3, (t_null is null OR si is null OR i < 0) as multi_and_col from vectortab2k_orc_null
+        where pmod(i,4) = 2
+        order by t_null, si, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+POSTHOOK: query: -- With some filtering
+SELECT sum(hash(*)) FROM
+    (SELECT t_null, si, i, (t_null is null) as child1, (si is null) as child2, (i < 0) as child3, (t_null is null OR si is null OR i < 0) as multi_and_col from vectortab2k_orc_null
+        where pmod(i,4) = 2
+        order by t_null, si, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+c0
+-13477536069
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_null, si, i, (t_null < 0) as child1, (si > 0) as child2, (i is null) as child3, (t_null < 0 OR si > 0 OR i is null) as multi_and_col from vectortab2k_orc_null
+        where pmod(i,4) = 3
+        order by t_null, si, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_null, si, i, (t_null < 0) as child1, (si > 0) as child2, (i is null) as child3, (t_null < 0 OR si > 0 OR i is null) as multi_and_col from vectortab2k_orc_null
+        where pmod(i,4) = 3
+        order by t_null, si, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+c0
+-48784614557
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_null, si_null, i_null, (t_null < 0) as child1, (si > 0) as child2, (i_null < 0) as child3, (t_null < 0 OR si > 0 OR i_null < 0) as multi_and_col from vectortab2k_orc_null
+        where pmod(i,4) = 0
+        order by t_null, si_null, i_null) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_null, si_null, i_null, (t_null < 0) as child1, (si > 0) as child2, (i_null < 0) as child3, (t_null < 0 OR si > 0 OR i_null < 0) as multi_and_col from vectortab2k_orc_null
+        where pmod(i,4) = 0
+        order by t_null, si_null, i_null) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+c0
+223184
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si_null, i_null, (t < 0) as child1, (si_null > 0) as child2, (i_null < 0) as child3, (t < 0 OR si_null > 0 OR i_null < 0) as multi_and_col from vectortab2k_orc_null
+        where pmod(i,4) != 2
+        order by t, si_null, i_null) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si_null, i_null, (t < 0) as child1, (si_null > 0) as child2, (i_null < 0) as child3, (t < 0 OR si_null > 0 OR i_null < 0) as multi_and_col from vectortab2k_orc_null
+        where pmod(i,4) != 2
+        order by t, si_null, i_null) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+c0
+6738993743


[4/5] hive git commit: HIVE-13084: Vectorization add support for PROJECTION Multi-AND/OR (Matt McCline, reviewed by Sergey Shelukhin)

Posted by mm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/9dd70f0d/ql/src/test/queries/clientpositive/vector_multi_or_projection.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/vector_multi_or_projection.q b/ql/src/test/queries/clientpositive/vector_multi_or_projection.q
new file mode 100644
index 0000000..0b680b3
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/vector_multi_or_projection.q
@@ -0,0 +1,198 @@
+set hive.cli.print.header=true;
+set hive.explain.user=false;
+SET hive.auto.convert.join=true;
+set hive.fetch.task.conversion=none;
+set hive.mapred.mode=nonstrict;
+
+-- SORT_QUERY_RESULTS
+
+create table vectortab2k(
+            t tinyint,
+            si smallint,
+            i int,
+            b bigint,
+            f float,
+            d double,
+            dc decimal(38,18),
+            bo boolean,
+            s string,
+            s2 string,
+            ts timestamp,
+            ts2 timestamp,
+            dt date)
+ROW FORMAT DELIMITED FIELDS TERMINATED BY '|'
+STORED AS TEXTFILE;
+
+LOAD DATA LOCAL INPATH '../../data/files/vectortab2k' OVERWRITE INTO TABLE vectortab2k;
+
+CREATE TABLE scratch AS SELECT t, si, i, b, bo FROM vectortab2k;
+INSERT INTO TABLE scratch VALUES (NULL, NULL, NULL, NULL, NULL);
+
+CREATE TABLE vectortab2k_orc STORED AS ORC AS SELECT * FROM scratch;
+
+SET hive.vectorized.execution.enabled=true;
+
+EXPLAIN
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t < 0 OR si > 0 OR i < 0) as multi_or_col from vectortab2k_orc
+        order by t, si, i) as q;
+
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t < 0 OR si > 0 OR i < 0) as multi_or_col from vectortab2k_orc
+        order by t, si, i) as q;
+
+EXPLAIN
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, b, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 OR si > 0 OR i < 0 OR b > 0) as multi_or_col from vectortab2k_orc
+        order by t, si, i, b) as q;
+
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, b, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 OR si > 0 OR i < 0 OR b > 0) as multi_or_col from vectortab2k_orc
+        order by t, si, i, b) as q;
+
+-- Use a boolean column rather than a column comparison expression.
+EXPLAIN
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo, b, (t < 0) as child1, (si > 0) as child2, bo as child3, (b > 0) as child4, (t < 0 OR si > 0 OR bo OR b > 0) as multi_or_col from vectortab2k_orc
+        order by t, si, bo, b) as q;
+
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo, b, (t < 0) as child1, (si > 0) as child2, bo as child3, (b > 0) as child4, (t < 0 OR si > 0 OR bo OR b > 0) as multi_or_col from vectortab2k_orc
+        order by t, si, bo, b) as q;
+
+-- With some filtering
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t < 0 OR si > 0 OR i < 0) as multi_or_col from vectortab2k_orc
+        where pmod(i,4) = 2
+        order by t, si, i) as q;
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, b, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 OR si > 0 OR i < 0 OR b > 0) as multi_or_col from vectortab2k_orc
+        where pmod(si,4) < 2
+        order by t, si, i, b) as q;
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo, b, (t < 0) as child1, (si > 0) as child2, bo as child3, (b > 0) as child4, (t < 0 OR si > 0 OR bo OR b > 0) as multi_or_col from vectortab2k_orc
+        where pmod(i,4) = 2
+        order by t, si, bo, b) as q;
+
+
+SET hive.vectorized.execution.enabled=false;
+
+CREATE TABLE scratch_repeat AS SELECT t, si, i, b, bo,
+    20 as t_repeat, 9000 as si_repeat, 9233320 as i_repeat, -823823999339992 as b_repeat, false as bo_repeat_false, true as bo_repeat_true FROM vectortab2k;
+
+-- The repeated columns ought to create repeated VectorizedRowBatch for those columns.
+-- And then when we do a comparison, we should generate a repeated boolean result.
+CREATE TABLE vectortab2k_orc_repeat STORED AS ORC AS SELECT * FROM scratch_repeat;
+
+SET hive.vectorized.execution.enabled=true;
+
+-- t_repeat < 0 should generate all false.
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, i, (t_repeat < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_repeat < 0 OR si > 0 OR i < 0) as multi_or_col from vectortab2k_orc_repeat
+        order by t_repeat, si, i) as q;
+
+-- t_repeat > 0 should generate all true.
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, i, (t_repeat > 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_repeat > 0 OR si > 0 OR i < 0) as multi_or_col from vectortab2k_orc_repeat
+        order by t_repeat, si, i) as q;
+
+-- Two repeated false columns at beginning...
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si_repeat, i, (t_repeat < 0) as child1, (si_repeat < 0) as child2, (i < 0) as child3, (t_repeat < 0 OR si_repeat < 0 OR i < 0) as multi_or_col from vectortab2k_orc_repeat
+        order by t_repeat, si_repeat, i) as q;
+
+-- si_repeat > 0 should generate all true.
+SELECT sum(hash(*)) FROM
+    (SELECT t, si_repeat, i, b, (t < 0) as child1, (si_repeat > 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 OR si_repeat > 0 OR i < 0 OR b > 0) as multi_or_col from vectortab2k_orc_repeat
+        order by t, si_repeat, i, b) as q;
+
+-- si_repeat < 0 should generate all false.
+SELECT sum(hash(*)) FROM
+    (SELECT t, si_repeat, i, b, (t < 0) as child1, (si_repeat < 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 OR si_repeat < 0 OR i < 0 OR b > 0) as multi_or_col from vectortab2k_orc_repeat
+        order by t, si_repeat, i, b) as q;
+
+-- Use a boolean column rather than a column comparison expression.
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, bo_repeat_false, b, (t_repeat > 0) as child1, (si > 0) as child2, (bo_repeat_false) as child3, (b > 0) as child4, (t_repeat > 0 OR si > 0 OR bo_repeat_false OR b > 0) as multi_or_col from vectortab2k_orc_repeat
+        order by t_repeat, si, bo_repeat_false, b) as q;
+
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, bo_repeat_true, b, (t_repeat > 0) as child1, (si > 0) as child2, (bo_repeat_true) as child3, (b > 0) as child4, (t_repeat > 0 OR si > 0 OR bo_repeat_true OR b > 0) as multi_or_col from vectortab2k_orc_repeat
+        order by t_repeat, si, bo_repeat_true, b) as q;
+
+-- With some filtering
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, i, (t_repeat < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_repeat < 0 OR si > 0 OR i < 0) as multi_or_col from vectortab2k_orc_repeat
+        where pmod(i,4) > 1
+        order by t_repeat, si, i) as q;
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, i, (t_repeat > 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_repeat > 0 OR si > 0 OR i < 0) as multi_or_col from vectortab2k_orc_repeat
+        where pmod(si,4) > 1
+        order by t_repeat, si, i) as q;
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si_repeat, i, (t_repeat < 0) as child1, (si_repeat < 0) as child2, (i < 0) as child3, (t_repeat < 0 OR si_repeat < 0 OR i < 0) as multi_or_col from vectortab2k_orc_repeat
+        where pmod(i,4) > 1
+        order by t_repeat, si_repeat, i) as q;
+SELECT sum(hash(*)) FROM
+    (SELECT t, si_repeat, i, b, (t < 0) as child1, (si_repeat > 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 OR si_repeat > 0 OR i < 0 OR b > 0) as multi_or_col from vectortab2k_orc_repeat
+        where pmod(t,4) > 1
+        order by t, si_repeat, i, b) as q;
+SELECT sum(hash(*)) FROM
+    (SELECT t, si_repeat, i, b, (t < 0) as child1, (si_repeat < 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 OR si_repeat < 0 OR i < 0 OR b > 0) as multi_or_col from vectortab2k_orc_repeat
+        where pmod(b,4) > 1
+        order by t, si_repeat, i, b) as q;
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, bo_repeat_false, b, (t_repeat > 0) as child1, (si > 0) as child2, (bo_repeat_false) as child3, (b > 0) as child4, (t_repeat > 0 OR si > 0 OR bo_repeat_false OR b > 0) as multi_or_col from vectortab2k_orc_repeat
+        where pmod(si,4) > 1
+        order by t_repeat, si, bo_repeat_false, b) as q;
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, bo_repeat_true, b, (t_repeat > 0) as child1, (si > 0) as child2, (bo_repeat_true) as child3, (b > 0) as child4, (t_repeat > 0 OR si > 0 OR bo_repeat_true OR b > 0) as multi_or_col from vectortab2k_orc_repeat
+        where pmod(si,4) < 2
+        order by t_repeat, si, bo_repeat_true, b) as q;
+
+
+SET hive.vectorized.execution.enabled=false;
+
+CREATE TABLE scratch_null AS SELECT t, si, i, b, bo,
+     cast(null as tinyint) as t_null, cast(null as smallint) as si_null, cast(null as int) as i_null, cast(null as bigint) as b_null, cast(null as boolean) as bo_null FROM vectortab2k;
+
+-- The nulled columns ought to create repeated null VectorizedRowBatch for those columns.
+CREATE TABLE vectortab2k_orc_null STORED AS ORC AS SELECT * FROM scratch_null;
+
+SET hive.vectorized.execution.enabled=true;
+
+
+SELECT sum(hash(*)) FROM
+    (SELECT t_null, si, i, (t_null is null) as child1, (si is null) as child2, (i < 0) as child3, (t_null is null OR si is null OR i < 0) as multi_and_col from vectortab2k_orc_null
+        order by t_null, si, i) as q;
+
+SELECT sum(hash(*)) FROM
+    (SELECT t_null, si, i, (t_null < 0) as child1, (si > 0) as child2, (i is null) as child3, (t_null < 0 OR si > 0 OR i is null) as multi_and_col from vectortab2k_orc_null
+        order by t_null, si, i) as q;
+
+SELECT sum(hash(*)) FROM
+    (SELECT t_null, si, i_null, (t_null < 0) as child1, (si > 0) as child2, (i_null < 0) as child3, (t_null < 0 OR si > 0 OR i_null < 0) as multi_and_col from vectortab2k_orc_null
+        order by t_null, si, i_null) as q;
+    
+SELECT sum(hash(*)) FROM
+    (SELECT t, si_null, i_null, (t < 0) as child1, (si_null > 0) as child2, (i_null < 0) as child3, (t < 0 OR si_null > 0 OR i_null < 0) as multi_and_col from vectortab2k_orc_null
+        order by t, si_null, i_null) as q;
+
+
+-- With some filtering
+SELECT sum(hash(*)) FROM
+    (SELECT t_null, si, i, (t_null is null) as child1, (si is null) as child2, (i < 0) as child3, (t_null is null OR si is null OR i < 0) as multi_and_col from vectortab2k_orc_null
+        where pmod(i,4) = 2
+        order by t_null, si, i) as q;
+SELECT sum(hash(*)) FROM
+    (SELECT t_null, si, i, (t_null < 0) as child1, (si > 0) as child2, (i is null) as child3, (t_null < 0 OR si > 0 OR i is null) as multi_and_col from vectortab2k_orc_null
+        where pmod(i,4) = 3
+        order by t_null, si, i) as q;
+SELECT sum(hash(*)) FROM
+    (SELECT t_null, si_null, i_null, (t_null < 0) as child1, (si > 0) as child2, (i_null < 0) as child3, (t_null < 0 OR si > 0 OR i_null < 0) as multi_and_col from vectortab2k_orc_null
+        where pmod(i,4) = 0
+        order by t_null, si_null, i_null) as q;
+SELECT sum(hash(*)) FROM
+    (SELECT t, si_null, i_null, (t < 0) as child1, (si_null > 0) as child2, (i_null < 0) as child3, (t < 0 OR si_null > 0 OR i_null < 0) as multi_and_col from vectortab2k_orc_null
+        where pmod(i,4) != 2
+        order by t, si_null, i_null) as q;

http://git-wip-us.apache.org/repos/asf/hive/blob/9dd70f0d/ql/src/test/results/clientpositive/tez/vector_multi_and_projection.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/vector_multi_and_projection.q.out b/ql/src/test/results/clientpositive/tez/vector_multi_and_projection.q.out
new file mode 100644
index 0000000..a9bf8fb
--- /dev/null
+++ b/ql/src/test/results/clientpositive/tez/vector_multi_and_projection.q.out
@@ -0,0 +1,821 @@
+PREHOOK: query: -- SORT_QUERY_RESULTS
+
+create table vectortab2k(
+            t tinyint,
+            si smallint,
+            i int,
+            b bigint,
+            f float,
+            d double,
+            dc decimal(38,18),
+            bo boolean,
+            s string,
+            s2 string,
+            ts timestamp,
+            ts2 timestamp,
+            dt date)
+ROW FORMAT DELIMITED FIELDS TERMINATED BY '|'
+STORED AS TEXTFILE
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@vectortab2k
+POSTHOOK: query: -- SORT_QUERY_RESULTS
+
+create table vectortab2k(
+            t tinyint,
+            si smallint,
+            i int,
+            b bigint,
+            f float,
+            d double,
+            dc decimal(38,18),
+            bo boolean,
+            s string,
+            s2 string,
+            ts timestamp,
+            ts2 timestamp,
+            dt date)
+ROW FORMAT DELIMITED FIELDS TERMINATED BY '|'
+STORED AS TEXTFILE
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@vectortab2k
+PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/vectortab2k' OVERWRITE INTO TABLE vectortab2k
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: default@vectortab2k
+POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/vectortab2k' OVERWRITE INTO TABLE vectortab2k
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: default@vectortab2k
+PREHOOK: query: CREATE TABLE scratch AS SELECT t, si, i, b, bo FROM vectortab2k
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@vectortab2k
+PREHOOK: Output: database:default
+PREHOOK: Output: default@scratch
+POSTHOOK: query: CREATE TABLE scratch AS SELECT t, si, i, b, bo FROM vectortab2k
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@vectortab2k
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@scratch
+POSTHOOK: Lineage: scratch.b SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: scratch.bo SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: scratch.i SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: scratch.si SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: scratch.t SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:t, type:tinyint, comment:null), ]
+t	si	i	b	bo
+PREHOOK: query: INSERT INTO TABLE scratch VALUES (NULL, NULL, NULL, NULL, NULL)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__1
+PREHOOK: Output: default@scratch
+POSTHOOK: query: INSERT INTO TABLE scratch VALUES (NULL, NULL, NULL, NULL, NULL)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__1
+POSTHOOK: Output: default@scratch
+POSTHOOK: Lineage: scratch.b EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col4, type:string, comment:), ]
+POSTHOOK: Lineage: scratch.bo EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col5, type:string, comment:), ]
+POSTHOOK: Lineage: scratch.i EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col3, type:string, comment:), ]
+POSTHOOK: Lineage: scratch.si EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+POSTHOOK: Lineage: scratch.t EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+_col0	_col1	_col2	_col3	_col4
+PREHOOK: query: CREATE TABLE vectortab2k_orc STORED AS ORC AS SELECT * FROM scratch
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@scratch
+PREHOOK: Output: database:default
+PREHOOK: Output: default@vectortab2k_orc
+POSTHOOK: query: CREATE TABLE vectortab2k_orc STORED AS ORC AS SELECT * FROM scratch
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@scratch
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@vectortab2k_orc
+POSTHOOK: Lineage: vectortab2k_orc.b SIMPLE [(scratch)scratch.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc.bo SIMPLE [(scratch)scratch.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc.i SIMPLE [(scratch)scratch.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc.si SIMPLE [(scratch)scratch.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc.t SIMPLE [(scratch)scratch.FieldSchema(name:t, type:tinyint, comment:null), ]
+scratch.t	scratch.si	scratch.i	scratch.b	scratch.bo
+PREHOOK: query: EXPLAIN
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t < 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc
+        order by t, si, i) as q
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t < 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc
+        order by t, si, i) as q
+POSTHOOK: type: QUERY
+Explain
+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:
+        Reducer 2 <- Map 1 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: vectortab2k_orc
+                  Statistics: Num rows: 2001 Data size: 45620 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: t (type: tinyint), si (type: smallint), i (type: int), (t < 0) (type: boolean), (si > 0) (type: boolean), (i < 0) (type: boolean), ((t < 0) and (si > 0) and (i < 0)) (type: boolean)
+                    outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
+                    Statistics: Num rows: 2001 Data size: 45620 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: int)
+                      sort order: +++
+                      Statistics: Num rows: 2001 Data size: 45620 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col3 (type: boolean), _col4 (type: boolean), _col5 (type: boolean), _col6 (type: boolean)
+            Execution mode: vectorized
+        Reducer 2 
+            Reduce Operator Tree:
+              Select Operator
+                expressions: hash(KEY.reducesinkkey0,KEY.reducesinkkey1,KEY.reducesinkkey2,VALUE._col0,VALUE._col1,VALUE._col2,VALUE._col3) (type: int)
+                outputColumnNames: _col0
+                Statistics: Num rows: 2001 Data size: 45620 Basic stats: COMPLETE Column stats: NONE
+                Group By Operator
+                  aggregations: sum(_col0)
+                  mode: complete
+                  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.SequenceFileInputFormat
+                        output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t < 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc
+        order by t, si, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t < 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc
+        order by t, si, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc
+#### A masked pattern was here ####
+c0
+-64411366206
+PREHOOK: query: EXPLAIN
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, b, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND i < 0 AND b > 0) as multi_and_col from vectortab2k_orc
+        order by t, si, i, b) as q
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, b, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND i < 0 AND b > 0) as multi_and_col from vectortab2k_orc
+        order by t, si, i, b) as q
+POSTHOOK: type: QUERY
+Explain
+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:
+        Reducer 2 <- Map 1 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: vectortab2k_orc
+                  Statistics: Num rows: 2001 Data size: 45620 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: t (type: tinyint), si (type: smallint), i (type: int), b (type: bigint), (t < 0) (type: boolean), (si > 0) (type: boolean), (i < 0) (type: boolean), (b > 0) (type: boolean), ((t < 0) and (si > 0) and (i < 0) and (b > 0)) (type: boolean)
+                    outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
+                    Statistics: Num rows: 2001 Data size: 45620 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint)
+                      sort order: ++++
+                      Statistics: Num rows: 2001 Data size: 45620 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col4 (type: boolean), _col5 (type: boolean), _col6 (type: boolean), _col7 (type: boolean), _col8 (type: boolean)
+            Execution mode: vectorized
+        Reducer 2 
+            Reduce Operator Tree:
+              Select Operator
+                expressions: hash(KEY.reducesinkkey0,KEY.reducesinkkey1,KEY.reducesinkkey2,KEY.reducesinkkey3,VALUE._col0,VALUE._col1,VALUE._col2,VALUE._col3,VALUE._col4) (type: int)
+                outputColumnNames: _col0
+                Statistics: Num rows: 2001 Data size: 45620 Basic stats: COMPLETE Column stats: NONE
+                Group By Operator
+                  aggregations: sum(_col0)
+                  mode: complete
+                  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.SequenceFileInputFormat
+                        output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, b, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND i < 0 AND b > 0) as multi_and_col from vectortab2k_orc
+        order by t, si, i, b) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, b, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND i < 0 AND b > 0) as multi_and_col from vectortab2k_orc
+        order by t, si, i, b) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc
+#### A masked pattern was here ####
+c0
+96890477236
+PREHOOK: query: -- Use a boolean column rather than a column comparison expression.
+EXPLAIN
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo, b, (t < 0) as child1, (si > 0) as child2, (bo) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND bo AND b > 0) as multi_and_col from vectortab2k_orc
+        order by t, si, bo, b) as q
+PREHOOK: type: QUERY
+POSTHOOK: query: -- Use a boolean column rather than a column comparison expression.
+EXPLAIN
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo, b, (t < 0) as child1, (si > 0) as child2, (bo) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND bo AND b > 0) as multi_and_col from vectortab2k_orc
+        order by t, si, bo, b) as q
+POSTHOOK: type: QUERY
+Explain
+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:
+        Reducer 2 <- Map 1 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: vectortab2k_orc
+                  Statistics: Num rows: 2001 Data size: 45620 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: t (type: tinyint), si (type: smallint), bo (type: boolean), b (type: bigint), (t < 0) (type: boolean), (si > 0) (type: boolean), (b > 0) (type: boolean), ((t < 0) and (si > 0) and bo and (b > 0)) (type: boolean)
+                    outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col7, _col8
+                    Statistics: Num rows: 2001 Data size: 45620 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: boolean), _col3 (type: bigint)
+                      sort order: ++++
+                      Statistics: Num rows: 2001 Data size: 45620 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col4 (type: boolean), _col5 (type: boolean), _col7 (type: boolean), _col8 (type: boolean)
+            Execution mode: vectorized
+        Reducer 2 
+            Reduce Operator Tree:
+              Select Operator
+                expressions: hash(KEY.reducesinkkey0,KEY.reducesinkkey1,KEY.reducesinkkey2,KEY.reducesinkkey3,VALUE._col0,VALUE._col1,KEY.reducesinkkey2,VALUE._col2,VALUE._col3) (type: int)
+                outputColumnNames: _col0
+                Statistics: Num rows: 2001 Data size: 45620 Basic stats: COMPLETE Column stats: NONE
+                Group By Operator
+                  aggregations: sum(_col0)
+                  mode: complete
+                  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.SequenceFileInputFormat
+                        output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo, b, (t < 0) as child1, (si > 0) as child2, (bo) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND bo AND b > 0) as multi_and_col from vectortab2k_orc
+        order by t, si, bo, b) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo, b, (t < 0) as child1, (si > 0) as child2, (bo) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND bo AND b > 0) as multi_and_col from vectortab2k_orc
+        order by t, si, bo, b) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc
+#### A masked pattern was here ####
+c0
+26141424353
+PREHOOK: query: -- With some filtering
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t < 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc
+        where pmod(t, 4) > 1
+        order by t, si, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc
+#### A masked pattern was here ####
+POSTHOOK: query: -- With some filtering
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t < 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc
+        where pmod(t, 4) > 1
+        order by t, si, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc
+#### A masked pattern was here ####
+c0
+-28501090956
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, b, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND i < 0 AND b > 0) as multi_and_col from vectortab2k_orc
+        where pmod(t, 4) < 2
+        order by t, si, i, b) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, b, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND i < 0 AND b > 0) as multi_and_col from vectortab2k_orc
+        where pmod(t, 4) < 2
+        order by t, si, i, b) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc
+#### A masked pattern was here ####
+c0
+38348662460
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo, b, (t < 0) as child1, (si > 0) as child2, (bo) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND bo AND b > 0) as multi_and_col from vectortab2k_orc
+        where pmod(b, 8) == 7
+        order by t, si, bo, b) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo, b, (t < 0) as child1, (si > 0) as child2, (bo) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND bo AND b > 0) as multi_and_col from vectortab2k_orc
+        where pmod(b, 8) == 7
+        order by t, si, bo, b) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc
+#### A masked pattern was here ####
+c0
+5176369462
+PREHOOK: query: CREATE TABLE scratch_repeat AS SELECT t, si, i, b, bo, 20 as t_repeat,
+     9000 as si_repeat, 9233320 as i_repeat, -823823999339992 as b_repeat, false as bo_repeat_false, true as bo_repeat_true FROM vectortab2k
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@vectortab2k
+PREHOOK: Output: database:default
+PREHOOK: Output: default@scratch_repeat
+POSTHOOK: query: CREATE TABLE scratch_repeat AS SELECT t, si, i, b, bo, 20 as t_repeat,
+     9000 as si_repeat, 9233320 as i_repeat, -823823999339992 as b_repeat, false as bo_repeat_false, true as bo_repeat_true FROM vectortab2k
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@vectortab2k
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@scratch_repeat
+POSTHOOK: Lineage: scratch_repeat.b SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: scratch_repeat.b_repeat SIMPLE []
+POSTHOOK: Lineage: scratch_repeat.bo SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: scratch_repeat.bo_repeat_false SIMPLE []
+POSTHOOK: Lineage: scratch_repeat.bo_repeat_true SIMPLE []
+POSTHOOK: Lineage: scratch_repeat.i SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: scratch_repeat.i_repeat SIMPLE []
+POSTHOOK: Lineage: scratch_repeat.si SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: scratch_repeat.si_repeat SIMPLE []
+POSTHOOK: Lineage: scratch_repeat.t SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:t, type:tinyint, comment:null), ]
+POSTHOOK: Lineage: scratch_repeat.t_repeat SIMPLE []
+t	si	i	b	bo	t_repeat	si_repeat	i_repeat	b_repeat	bo_repeat_false	bo_repeat_true
+PREHOOK: query: -- The repeated columns ought to create repeated VectorizedRowBatch for those columns.
+-- And then when we do a comparison, we should generate a repeated boolean result.
+CREATE TABLE vectortab2k_orc_repeat STORED AS ORC AS SELECT * FROM scratch_repeat
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@scratch_repeat
+PREHOOK: Output: database:default
+PREHOOK: Output: default@vectortab2k_orc_repeat
+POSTHOOK: query: -- The repeated columns ought to create repeated VectorizedRowBatch for those columns.
+-- And then when we do a comparison, we should generate a repeated boolean result.
+CREATE TABLE vectortab2k_orc_repeat STORED AS ORC AS SELECT * FROM scratch_repeat
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@scratch_repeat
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@vectortab2k_orc_repeat
+POSTHOOK: Lineage: vectortab2k_orc_repeat.b SIMPLE [(scratch_repeat)scratch_repeat.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_repeat.b_repeat SIMPLE [(scratch_repeat)scratch_repeat.FieldSchema(name:b_repeat, type:bigint, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_repeat.bo SIMPLE [(scratch_repeat)scratch_repeat.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_repeat.bo_repeat_false SIMPLE [(scratch_repeat)scratch_repeat.FieldSchema(name:bo_repeat_false, type:boolean, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_repeat.bo_repeat_true SIMPLE [(scratch_repeat)scratch_repeat.FieldSchema(name:bo_repeat_true, type:boolean, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_repeat.i SIMPLE [(scratch_repeat)scratch_repeat.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_repeat.i_repeat SIMPLE [(scratch_repeat)scratch_repeat.FieldSchema(name:i_repeat, type:int, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_repeat.si SIMPLE [(scratch_repeat)scratch_repeat.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_repeat.si_repeat SIMPLE [(scratch_repeat)scratch_repeat.FieldSchema(name:si_repeat, type:int, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_repeat.t SIMPLE [(scratch_repeat)scratch_repeat.FieldSchema(name:t, type:tinyint, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_repeat.t_repeat SIMPLE [(scratch_repeat)scratch_repeat.FieldSchema(name:t_repeat, type:int, comment:null), ]
+scratch_repeat.t	scratch_repeat.si	scratch_repeat.i	scratch_repeat.b	scratch_repeat.bo	scratch_repeat.t_repeat	scratch_repeat.si_repeat	scratch_repeat.i_repeat	scratch_repeat.b_repeat	scratch_repeat.bo_repeat_false	scratch_repeat.bo_repeat_true
+PREHOOK: query: -- t_repeat > 0 should generate all true.
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, i, (t_repeat > 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_repeat > 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc_repeat
+        order by t_repeat, si, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: -- t_repeat > 0 should generate all true.
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, i, (t_repeat > 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_repeat > 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc_repeat
+        order by t_repeat, si, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+-53648296618
+PREHOOK: query: -- t_repeat < 0 should generate all false.
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, i, (t_repeat < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_repeat < 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc_repeat
+        order by t_repeat, si, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: -- t_repeat < 0 should generate all false.
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, i, (t_repeat < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_repeat < 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc_repeat
+        order by t_repeat, si, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+-53707879050
+PREHOOK: query: -- Two repeated false columns at beginning...
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si_repeat, i, (t_repeat > 0) as child1, (si_repeat > 0) as child2, (i < 0) as child3, (t_repeat > 0 AND si_repeat > 0 AND i < 0) as multi_and_col from vectortab2k_orc_repeat
+        order by t_repeat, si_repeat, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: -- Two repeated false columns at beginning...
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si_repeat, i, (t_repeat > 0) as child1, (si_repeat > 0) as child2, (i < 0) as child3, (t_repeat > 0 AND si_repeat > 0 AND i < 0) as multi_and_col from vectortab2k_orc_repeat
+        order by t_repeat, si_repeat, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+-55121400269
+PREHOOK: query: -- si_repeat > 0 should generate all true.
+SELECT sum(hash(*)) FROM
+    (SELECT t, si_repeat, i, b_repeat, (t < 0 ) as child1, (si_repeat > 0) as child2, (i < 0) as child3, (b_repeat > 0) as child4, (t < 0 AND si_repeat > 0 AND i < 0 AND b_repeat > 0) as multi_and_col from vectortab2k_orc_repeat
+        order by t, si_repeat, i, b_repeat) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: -- si_repeat > 0 should generate all true.
+SELECT sum(hash(*)) FROM
+    (SELECT t, si_repeat, i, b_repeat, (t < 0 ) as child1, (si_repeat > 0) as child2, (i < 0) as child3, (b_repeat > 0) as child4, (t < 0 AND si_repeat > 0 AND i < 0 AND b_repeat > 0) as multi_and_col from vectortab2k_orc_repeat
+        order by t, si_repeat, i, b_repeat) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+-39803959564
+PREHOOK: query: -- si_repeat < 0 should generate all false.
+SELECT sum(hash(*)) FROM
+    (SELECT t, si_repeat, i, b_repeat, (t < 0) as child1, (si_repeat < 0) as child2, (i < 0) as child3, (b_repeat > 0) as child4, (t < 0 AND si_repeat < 0 AND i < 0 AND b_repeat > 0) as multi_and_col from vectortab2k_orc_repeat
+        order by t, si_repeat, i, b_repeat) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: -- si_repeat < 0 should generate all false.
+SELECT sum(hash(*)) FROM
+    (SELECT t, si_repeat, i, b_repeat, (t < 0) as child1, (si_repeat < 0) as child2, (i < 0) as child3, (b_repeat > 0) as child4, (t < 0 AND si_repeat < 0 AND i < 0 AND b_repeat > 0) as multi_and_col from vectortab2k_orc_repeat
+        order by t, si_repeat, i, b_repeat) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+-39863541564
+PREHOOK: query: -- Use a boolean column rather than a column comparison expression.
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo_repeat_false, b, (t < 0) as child1, (si > 0) as child2, (bo_repeat_false) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND bo_repeat_false AND b > 0) as multi_and_col from vectortab2k_orc_repeat
+        order by t, si, bo_repeat_false, b) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: -- Use a boolean column rather than a column comparison expression.
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo_repeat_false, b, (t < 0) as child1, (si > 0) as child2, (bo_repeat_false) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND bo_repeat_false AND b > 0) as multi_and_col from vectortab2k_orc_repeat
+        order by t, si, bo_repeat_false, b) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+56950496526
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo_repeat_true, b, (t < 0) as child1, (si > 0) as child2, (bo_repeat_true) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND bo_repeat_true AND b > 0) as multi_and_col from vectortab2k_orc_repeat
+        order by t, si, bo_repeat_true, b) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo_repeat_true, b, (t < 0) as child1, (si > 0) as child2, (bo_repeat_true) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND bo_repeat_true AND b > 0) as multi_and_col from vectortab2k_orc_repeat
+        order by t, si, bo_repeat_true, b) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+2303712725
+PREHOOK: query: -- With some filtering
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, i, (t_repeat > 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_repeat > 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc_repeat
+        where pmod(si, 4) = 0
+        order by t_repeat, si, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: -- With some filtering
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, i, (t_repeat > 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_repeat > 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc_repeat
+        where pmod(si, 4) = 0
+        order by t_repeat, si, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+-4381273667
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, i, (t_repeat < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_repeat < 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc_repeat
+        where pmod(si, 4) = 3
+        order by t_repeat, si, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, i, (t_repeat < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_repeat < 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc_repeat
+        where pmod(si, 4) = 3
+        order by t_repeat, si, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+-954303006
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si_repeat, i, (t_repeat > 0) as child1, (si_repeat > 0) as child2, (i < 0) as child3, (t_repeat > 0 AND si_repeat > 0 AND i < 0) as multi_and_col from vectortab2k_orc_repeat
+        where pmod(si, 4) != 3
+        order by t_repeat, si_repeat, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si_repeat, i, (t_repeat > 0) as child1, (si_repeat > 0) as child2, (i < 0) as child3, (t_repeat > 0 AND si_repeat > 0 AND i < 0) as multi_and_col from vectortab2k_orc_repeat
+        where pmod(si, 4) != 3
+        order by t_repeat, si_repeat, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+-45312591578
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si_repeat, i, b_repeat, (t < 0 ) as child1, (si_repeat > 0) as child2, (i < 0) as child3, (b_repeat > 0) as child4, (t < 0 AND si_repeat > 0 AND i < 0 AND b_repeat > 0) as multi_and_col from vectortab2k_orc_repeat
+        where pmod(si, 4) < 2
+        order by t, si_repeat, i, b_repeat) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si_repeat, i, b_repeat, (t < 0 ) as child1, (si_repeat > 0) as child2, (i < 0) as child3, (b_repeat > 0) as child4, (t < 0 AND si_repeat > 0 AND i < 0 AND b_repeat > 0) as multi_and_col from vectortab2k_orc_repeat
+        where pmod(si, 4) < 2
+        order by t, si_repeat, i, b_repeat) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+-30297068525
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si_repeat, i, b_repeat, (t < 0) as child1, (si_repeat < 0) as child2, (i < 0) as child3, (b_repeat > 0) as child4, (t < 0 AND si_repeat < 0 AND i < 0 AND b_repeat > 0) as multi_and_col from vectortab2k_orc_repeat
+        where pmod(t, 4) = 0
+        order by t, si_repeat, i, b_repeat) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si_repeat, i, b_repeat, (t < 0) as child1, (si_repeat < 0) as child2, (i < 0) as child3, (b_repeat > 0) as child4, (t < 0 AND si_repeat < 0 AND i < 0 AND b_repeat > 0) as multi_and_col from vectortab2k_orc_repeat
+        where pmod(t, 4) = 0
+        order by t, si_repeat, i, b_repeat) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+32929905326
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo_repeat_false, b, (t < 0) as child1, (si > 0) as child2, (bo_repeat_false) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND bo_repeat_false AND b > 0) as multi_and_col from vectortab2k_orc_repeat
+        where pmod(b, 4) > 1
+        order by t, si, bo_repeat_false, b) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo_repeat_false, b, (t < 0) as child1, (si > 0) as child2, (bo_repeat_false) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND bo_repeat_false AND b > 0) as multi_and_col from vectortab2k_orc_repeat
+        where pmod(b, 4) > 1
+        order by t, si, bo_repeat_false, b) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+10610127509
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo_repeat_true, b, (t < 0) as child1, (si > 0) as child2, (bo_repeat_true) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND bo_repeat_true AND b > 0) as multi_and_col from vectortab2k_orc_repeat
+        where pmod(si, 4) < 3
+        order by t, si, bo_repeat_true, b) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo_repeat_true, b, (t < 0) as child1, (si > 0) as child2, (bo_repeat_true) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND bo_repeat_true AND b > 0) as multi_and_col from vectortab2k_orc_repeat
+        where pmod(si, 4) < 3
+        order by t, si, bo_repeat_true, b) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+-16740409601
+PREHOOK: query: CREATE TABLE scratch_null AS SELECT t, si, i, b, bo,
+     cast(null as tinyint) as t_null, cast(null as smallint) as si_null, cast(null as int) as i_null, cast(null as bigint) as b_null, cast(null as boolean) as bo_null FROM vectortab2k
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@vectortab2k
+PREHOOK: Output: database:default
+PREHOOK: Output: default@scratch_null
+POSTHOOK: query: CREATE TABLE scratch_null AS SELECT t, si, i, b, bo,
+     cast(null as tinyint) as t_null, cast(null as smallint) as si_null, cast(null as int) as i_null, cast(null as bigint) as b_null, cast(null as boolean) as bo_null FROM vectortab2k
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@vectortab2k
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@scratch_null
+POSTHOOK: Lineage: scratch_null.b SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: scratch_null.b_null SIMPLE []
+POSTHOOK: Lineage: scratch_null.bo SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: scratch_null.bo_null SIMPLE []
+POSTHOOK: Lineage: scratch_null.i SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: scratch_null.i_null SIMPLE []
+POSTHOOK: Lineage: scratch_null.si SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: scratch_null.si_null SIMPLE []
+POSTHOOK: Lineage: scratch_null.t SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:t, type:tinyint, comment:null), ]
+POSTHOOK: Lineage: scratch_null.t_null SIMPLE []
+t	si	i	b	bo	t_null	si_null	i_null	b_null	bo_null
+PREHOOK: query: -- The nulled columns ought to create repeated null VectorizedRowBatch for those columns.
+CREATE TABLE vectortab2k_orc_null STORED AS ORC AS SELECT * FROM scratch_null
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@scratch_null
+PREHOOK: Output: database:default
+PREHOOK: Output: default@vectortab2k_orc_null
+POSTHOOK: query: -- The nulled columns ought to create repeated null VectorizedRowBatch for those columns.
+CREATE TABLE vectortab2k_orc_null STORED AS ORC AS SELECT * FROM scratch_null
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@scratch_null
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@vectortab2k_orc_null
+POSTHOOK: Lineage: vectortab2k_orc_null.b SIMPLE [(scratch_null)scratch_null.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_null.b_null SIMPLE [(scratch_null)scratch_null.FieldSchema(name:b_null, type:bigint, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_null.bo SIMPLE [(scratch_null)scratch_null.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_null.bo_null SIMPLE [(scratch_null)scratch_null.FieldSchema(name:bo_null, type:boolean, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_null.i SIMPLE [(scratch_null)scratch_null.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_null.i_null SIMPLE [(scratch_null)scratch_null.FieldSchema(name:i_null, type:int, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_null.si SIMPLE [(scratch_null)scratch_null.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_null.si_null SIMPLE [(scratch_null)scratch_null.FieldSchema(name:si_null, type:smallint, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_null.t SIMPLE [(scratch_null)scratch_null.FieldSchema(name:t, type:tinyint, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_null.t_null SIMPLE [(scratch_null)scratch_null.FieldSchema(name:t_null, type:tinyint, comment:null), ]
+scratch_null.t	scratch_null.si	scratch_null.i	scratch_null.b	scratch_null.bo	scratch_null.t_null	scratch_null.si_null	scratch_null.i_null	scratch_null.b_null	scratch_null.bo_null
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_null, si, i, (t_null is null) as child1, (si > 0) as child2, (i < 0) as child3, (t_null is null AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc_null
+        order by t_null, si, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_null, si, i, (t_null is null) as child1, (si > 0) as child2, (i < 0) as child3, (t_null is null AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc_null
+        order by t_null, si, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+c0
+-77365671658
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_null, si, i, (t_null < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_null < 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc_null
+        order by t_null, si, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_null, si, i, (t_null < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_null < 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc_null
+        order by t_null, si, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+c0
+-77425254090
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_null, si_null, i, (t_null is null) as child1, (si_null > 0) as child2, (i < 0) as child3, (t_null is null AND si_null > 0 AND i < 0) as multi_and_col from vectortab2k_orc_null
+        order by t_null, si_null, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_null, si_null, i, (t_null is null) as child1, (si_null > 0) as child2, (i < 0) as child3, (t_null is null AND si_null > 0 AND i < 0) as multi_and_col from vectortab2k_orc_null
+        order by t_null, si_null, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+c0
+-127742262531
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_null, si_null, i_null, (t_null is null) as child1, (si_null is null) as child2, (i_null is null) as child3, (t_null is null AND si_null is null AND i_null is null) as multi_and_col from vectortab2k_orc_null
+        order by t_null, si_null, i_null) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_null, si_null, i_null, (t_null is null) as child1, (si_null is null) as child2, (i_null is null) as child3, (t_null is null AND si_null is null AND i_null is null) as multi_and_col from vectortab2k_orc_null
+        order by t_null, si_null, i_null) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+c0
+61568000
+PREHOOK: query: -- With some filtering
+SELECT sum(hash(*)) FROM
+    (SELECT t_null, si, i, (t_null is null) as child1, (si > 0) as child2, (i < 0) as child3, (t_null is null AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc_null
+        where pmod(i,4) = 3
+        order by t_null, si, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+POSTHOOK: query: -- With some filtering
+SELECT sum(hash(*)) FROM
+    (SELECT t_null, si, i, (t_null is null) as child1, (si > 0) as child2, (i < 0) as child3, (t_null is null AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc_null
+        where pmod(i,4) = 3
+        order by t_null, si, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+c0
+-48771559290
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_null, si, i, (t_null < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_null < 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc_null
+        where pmod(i,4) = 2
+        order by t_null, si, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_null, si, i, (t_null < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_null < 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc_null
+        where pmod(i,4) = 2
+        order by t_null, si, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+c0
+-13491514290
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_null, si_null, i, (t_null is null) as child1, (si_null > 0) as child2, (i < 0) as child3, (t_null is null AND si_null > 0 AND i < 0) as multi_and_col from vectortab2k_orc_null
+        where pmod(i,4) != 3
+        order by t_null, si_null, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_null, si_null, i, (t_null is null) as child1, (si_null > 0) as child2, (i < 0) as child3, (t_null is null AND si_null > 0 AND i < 0) as multi_and_col from vectortab2k_orc_null
+        where pmod(i,4) != 3
+        order by t_null, si_null, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+c0
+-96004829404
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_null, si_null, i_null, (t_null is null) as child1, (si_null is null) as child2, (i_null is null) as child3, (t_null is null AND si_null is null AND i_null is null) as multi_and_col from vectortab2k_orc_null
+        where pmod(i,4) < 3
+        order by t_null, si_null, i_null) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_null, si_null, i_null, (t_null is null) as child1, (si_null is null) as child2, (i_null is null) as child3, (t_null is null AND si_null is null AND i_null is null) as multi_and_col from vectortab2k_orc_null
+        where pmod(i,4) < 3
+        order by t_null, si_null, i_null) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+c0
+45252480


[2/5] hive git commit: HIVE-13084: Vectorization add support for PROJECTION Multi-AND/OR (Matt McCline, reviewed by Sergey Shelukhin)

Posted by mm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/9dd70f0d/ql/src/test/results/clientpositive/vector_multi_and_projection.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_multi_and_projection.q.out b/ql/src/test/results/clientpositive/vector_multi_and_projection.q.out
new file mode 100644
index 0000000..15edf54
--- /dev/null
+++ b/ql/src/test/results/clientpositive/vector_multi_and_projection.q.out
@@ -0,0 +1,800 @@
+PREHOOK: query: -- SORT_QUERY_RESULTS
+
+create table vectortab2k(
+            t tinyint,
+            si smallint,
+            i int,
+            b bigint,
+            f float,
+            d double,
+            dc decimal(38,18),
+            bo boolean,
+            s string,
+            s2 string,
+            ts timestamp,
+            ts2 timestamp,
+            dt date)
+ROW FORMAT DELIMITED FIELDS TERMINATED BY '|'
+STORED AS TEXTFILE
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@vectortab2k
+POSTHOOK: query: -- SORT_QUERY_RESULTS
+
+create table vectortab2k(
+            t tinyint,
+            si smallint,
+            i int,
+            b bigint,
+            f float,
+            d double,
+            dc decimal(38,18),
+            bo boolean,
+            s string,
+            s2 string,
+            ts timestamp,
+            ts2 timestamp,
+            dt date)
+ROW FORMAT DELIMITED FIELDS TERMINATED BY '|'
+STORED AS TEXTFILE
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@vectortab2k
+PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/vectortab2k' OVERWRITE INTO TABLE vectortab2k
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: default@vectortab2k
+POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/vectortab2k' OVERWRITE INTO TABLE vectortab2k
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: default@vectortab2k
+PREHOOK: query: CREATE TABLE scratch AS SELECT t, si, i, b, bo FROM vectortab2k
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@vectortab2k
+PREHOOK: Output: database:default
+PREHOOK: Output: default@scratch
+POSTHOOK: query: CREATE TABLE scratch AS SELECT t, si, i, b, bo FROM vectortab2k
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@vectortab2k
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@scratch
+POSTHOOK: Lineage: scratch.b SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: scratch.bo SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: scratch.i SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: scratch.si SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: scratch.t SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:t, type:tinyint, comment:null), ]
+t	si	i	b	bo
+PREHOOK: query: INSERT INTO TABLE scratch VALUES (NULL, NULL, NULL, NULL, NULL)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__1
+PREHOOK: Output: default@scratch
+POSTHOOK: query: INSERT INTO TABLE scratch VALUES (NULL, NULL, NULL, NULL, NULL)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__1
+POSTHOOK: Output: default@scratch
+POSTHOOK: Lineage: scratch.b EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col4, type:string, comment:), ]
+POSTHOOK: Lineage: scratch.bo EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col5, type:string, comment:), ]
+POSTHOOK: Lineage: scratch.i EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col3, type:string, comment:), ]
+POSTHOOK: Lineage: scratch.si EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+POSTHOOK: Lineage: scratch.t EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+_col0	_col1	_col2	_col3	_col4
+PREHOOK: query: CREATE TABLE vectortab2k_orc STORED AS ORC AS SELECT * FROM scratch
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@scratch
+PREHOOK: Output: database:default
+PREHOOK: Output: default@vectortab2k_orc
+POSTHOOK: query: CREATE TABLE vectortab2k_orc STORED AS ORC AS SELECT * FROM scratch
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@scratch
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@vectortab2k_orc
+POSTHOOK: Lineage: vectortab2k_orc.b SIMPLE [(scratch)scratch.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc.bo SIMPLE [(scratch)scratch.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc.i SIMPLE [(scratch)scratch.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc.si SIMPLE [(scratch)scratch.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc.t SIMPLE [(scratch)scratch.FieldSchema(name:t, type:tinyint, comment:null), ]
+scratch.t	scratch.si	scratch.i	scratch.b	scratch.bo
+PREHOOK: query: EXPLAIN
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t < 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc
+        order by t, si, i) as q
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t < 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc
+        order by t, si, i) as q
+POSTHOOK: type: QUERY
+Explain
+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: vectortab2k_orc
+            Statistics: Num rows: 2001 Data size: 45620 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: t (type: tinyint), si (type: smallint), i (type: int), (t < 0) (type: boolean), (si > 0) (type: boolean), (i < 0) (type: boolean), ((t < 0) and (si > 0) and (i < 0)) (type: boolean)
+              outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
+              Statistics: Num rows: 2001 Data size: 45620 Basic stats: COMPLETE Column stats: NONE
+              Reduce Output Operator
+                key expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: int)
+                sort order: +++
+                Statistics: Num rows: 2001 Data size: 45620 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col3 (type: boolean), _col4 (type: boolean), _col5 (type: boolean), _col6 (type: boolean)
+      Execution mode: vectorized
+      Reduce Operator Tree:
+        Select Operator
+          expressions: hash(KEY.reducesinkkey0,KEY.reducesinkkey1,KEY.reducesinkkey2,VALUE._col0,VALUE._col1,VALUE._col2,VALUE._col3) (type: int)
+          outputColumnNames: _col0
+          Statistics: Num rows: 2001 Data size: 45620 Basic stats: COMPLETE Column stats: NONE
+          Group By Operator
+            aggregations: sum(_col0)
+            mode: complete
+            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.SequenceFileInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t < 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc
+        order by t, si, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t < 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc
+        order by t, si, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc
+#### A masked pattern was here ####
+c0
+-64411366206
+PREHOOK: query: EXPLAIN
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, b, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND i < 0 AND b > 0) as multi_and_col from vectortab2k_orc
+        order by t, si, i, b) as q
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, b, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND i < 0 AND b > 0) as multi_and_col from vectortab2k_orc
+        order by t, si, i, b) as q
+POSTHOOK: type: QUERY
+Explain
+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: vectortab2k_orc
+            Statistics: Num rows: 2001 Data size: 45620 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: t (type: tinyint), si (type: smallint), i (type: int), b (type: bigint), (t < 0) (type: boolean), (si > 0) (type: boolean), (i < 0) (type: boolean), (b > 0) (type: boolean), ((t < 0) and (si > 0) and (i < 0) and (b > 0)) (type: boolean)
+              outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
+              Statistics: Num rows: 2001 Data size: 45620 Basic stats: COMPLETE Column stats: NONE
+              Reduce Output Operator
+                key expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint)
+                sort order: ++++
+                Statistics: Num rows: 2001 Data size: 45620 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col4 (type: boolean), _col5 (type: boolean), _col6 (type: boolean), _col7 (type: boolean), _col8 (type: boolean)
+      Execution mode: vectorized
+      Reduce Operator Tree:
+        Select Operator
+          expressions: hash(KEY.reducesinkkey0,KEY.reducesinkkey1,KEY.reducesinkkey2,KEY.reducesinkkey3,VALUE._col0,VALUE._col1,VALUE._col2,VALUE._col3,VALUE._col4) (type: int)
+          outputColumnNames: _col0
+          Statistics: Num rows: 2001 Data size: 45620 Basic stats: COMPLETE Column stats: NONE
+          Group By Operator
+            aggregations: sum(_col0)
+            mode: complete
+            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.SequenceFileInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, b, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND i < 0 AND b > 0) as multi_and_col from vectortab2k_orc
+        order by t, si, i, b) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, b, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND i < 0 AND b > 0) as multi_and_col from vectortab2k_orc
+        order by t, si, i, b) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc
+#### A masked pattern was here ####
+c0
+96890477236
+PREHOOK: query: -- Use a boolean column rather than a column comparison expression.
+EXPLAIN
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo, b, (t < 0) as child1, (si > 0) as child2, (bo) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND bo AND b > 0) as multi_and_col from vectortab2k_orc
+        order by t, si, bo, b) as q
+PREHOOK: type: QUERY
+POSTHOOK: query: -- Use a boolean column rather than a column comparison expression.
+EXPLAIN
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo, b, (t < 0) as child1, (si > 0) as child2, (bo) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND bo AND b > 0) as multi_and_col from vectortab2k_orc
+        order by t, si, bo, b) as q
+POSTHOOK: type: QUERY
+Explain
+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: vectortab2k_orc
+            Statistics: Num rows: 2001 Data size: 45620 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: t (type: tinyint), si (type: smallint), bo (type: boolean), b (type: bigint), (t < 0) (type: boolean), (si > 0) (type: boolean), (b > 0) (type: boolean), ((t < 0) and (si > 0) and bo and (b > 0)) (type: boolean)
+              outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col7, _col8
+              Statistics: Num rows: 2001 Data size: 45620 Basic stats: COMPLETE Column stats: NONE
+              Reduce Output Operator
+                key expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: boolean), _col3 (type: bigint)
+                sort order: ++++
+                Statistics: Num rows: 2001 Data size: 45620 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col4 (type: boolean), _col5 (type: boolean), _col7 (type: boolean), _col8 (type: boolean)
+      Execution mode: vectorized
+      Reduce Operator Tree:
+        Select Operator
+          expressions: hash(KEY.reducesinkkey0,KEY.reducesinkkey1,KEY.reducesinkkey2,KEY.reducesinkkey3,VALUE._col0,VALUE._col1,KEY.reducesinkkey2,VALUE._col2,VALUE._col3) (type: int)
+          outputColumnNames: _col0
+          Statistics: Num rows: 2001 Data size: 45620 Basic stats: COMPLETE Column stats: NONE
+          Group By Operator
+            aggregations: sum(_col0)
+            mode: complete
+            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.SequenceFileInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo, b, (t < 0) as child1, (si > 0) as child2, (bo) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND bo AND b > 0) as multi_and_col from vectortab2k_orc
+        order by t, si, bo, b) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo, b, (t < 0) as child1, (si > 0) as child2, (bo) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND bo AND b > 0) as multi_and_col from vectortab2k_orc
+        order by t, si, bo, b) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc
+#### A masked pattern was here ####
+c0
+26141424353
+PREHOOK: query: -- With some filtering
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t < 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc
+        where pmod(t, 4) > 1
+        order by t, si, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc
+#### A masked pattern was here ####
+POSTHOOK: query: -- With some filtering
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t < 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc
+        where pmod(t, 4) > 1
+        order by t, si, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc
+#### A masked pattern was here ####
+c0
+-28501090956
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, b, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND i < 0 AND b > 0) as multi_and_col from vectortab2k_orc
+        where pmod(t, 4) < 2
+        order by t, si, i, b) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, b, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND i < 0 AND b > 0) as multi_and_col from vectortab2k_orc
+        where pmod(t, 4) < 2
+        order by t, si, i, b) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc
+#### A masked pattern was here ####
+c0
+38348662460
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo, b, (t < 0) as child1, (si > 0) as child2, (bo) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND bo AND b > 0) as multi_and_col from vectortab2k_orc
+        where pmod(b, 8) == 7
+        order by t, si, bo, b) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo, b, (t < 0) as child1, (si > 0) as child2, (bo) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND bo AND b > 0) as multi_and_col from vectortab2k_orc
+        where pmod(b, 8) == 7
+        order by t, si, bo, b) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc
+#### A masked pattern was here ####
+c0
+5176369462
+PREHOOK: query: CREATE TABLE scratch_repeat AS SELECT t, si, i, b, bo, 20 as t_repeat,
+     9000 as si_repeat, 9233320 as i_repeat, -823823999339992 as b_repeat, false as bo_repeat_false, true as bo_repeat_true FROM vectortab2k
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@vectortab2k
+PREHOOK: Output: database:default
+PREHOOK: Output: default@scratch_repeat
+POSTHOOK: query: CREATE TABLE scratch_repeat AS SELECT t, si, i, b, bo, 20 as t_repeat,
+     9000 as si_repeat, 9233320 as i_repeat, -823823999339992 as b_repeat, false as bo_repeat_false, true as bo_repeat_true FROM vectortab2k
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@vectortab2k
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@scratch_repeat
+POSTHOOK: Lineage: scratch_repeat.b SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: scratch_repeat.b_repeat SIMPLE []
+POSTHOOK: Lineage: scratch_repeat.bo SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: scratch_repeat.bo_repeat_false SIMPLE []
+POSTHOOK: Lineage: scratch_repeat.bo_repeat_true SIMPLE []
+POSTHOOK: Lineage: scratch_repeat.i SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: scratch_repeat.i_repeat SIMPLE []
+POSTHOOK: Lineage: scratch_repeat.si SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: scratch_repeat.si_repeat SIMPLE []
+POSTHOOK: Lineage: scratch_repeat.t SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:t, type:tinyint, comment:null), ]
+POSTHOOK: Lineage: scratch_repeat.t_repeat SIMPLE []
+t	si	i	b	bo	t_repeat	si_repeat	i_repeat	b_repeat	bo_repeat_false	bo_repeat_true
+PREHOOK: query: -- The repeated columns ought to create repeated VectorizedRowBatch for those columns.
+-- And then when we do a comparison, we should generate a repeated boolean result.
+CREATE TABLE vectortab2k_orc_repeat STORED AS ORC AS SELECT * FROM scratch_repeat
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@scratch_repeat
+PREHOOK: Output: database:default
+PREHOOK: Output: default@vectortab2k_orc_repeat
+POSTHOOK: query: -- The repeated columns ought to create repeated VectorizedRowBatch for those columns.
+-- And then when we do a comparison, we should generate a repeated boolean result.
+CREATE TABLE vectortab2k_orc_repeat STORED AS ORC AS SELECT * FROM scratch_repeat
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@scratch_repeat
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@vectortab2k_orc_repeat
+POSTHOOK: Lineage: vectortab2k_orc_repeat.b SIMPLE [(scratch_repeat)scratch_repeat.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_repeat.b_repeat SIMPLE [(scratch_repeat)scratch_repeat.FieldSchema(name:b_repeat, type:bigint, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_repeat.bo SIMPLE [(scratch_repeat)scratch_repeat.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_repeat.bo_repeat_false SIMPLE [(scratch_repeat)scratch_repeat.FieldSchema(name:bo_repeat_false, type:boolean, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_repeat.bo_repeat_true SIMPLE [(scratch_repeat)scratch_repeat.FieldSchema(name:bo_repeat_true, type:boolean, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_repeat.i SIMPLE [(scratch_repeat)scratch_repeat.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_repeat.i_repeat SIMPLE [(scratch_repeat)scratch_repeat.FieldSchema(name:i_repeat, type:int, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_repeat.si SIMPLE [(scratch_repeat)scratch_repeat.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_repeat.si_repeat SIMPLE [(scratch_repeat)scratch_repeat.FieldSchema(name:si_repeat, type:int, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_repeat.t SIMPLE [(scratch_repeat)scratch_repeat.FieldSchema(name:t, type:tinyint, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_repeat.t_repeat SIMPLE [(scratch_repeat)scratch_repeat.FieldSchema(name:t_repeat, type:int, comment:null), ]
+scratch_repeat.t	scratch_repeat.si	scratch_repeat.i	scratch_repeat.b	scratch_repeat.bo	scratch_repeat.t_repeat	scratch_repeat.si_repeat	scratch_repeat.i_repeat	scratch_repeat.b_repeat	scratch_repeat.bo_repeat_false	scratch_repeat.bo_repeat_true
+PREHOOK: query: -- t_repeat > 0 should generate all true.
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, i, (t_repeat > 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_repeat > 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc_repeat
+        order by t_repeat, si, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: -- t_repeat > 0 should generate all true.
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, i, (t_repeat > 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_repeat > 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc_repeat
+        order by t_repeat, si, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+-53648296618
+PREHOOK: query: -- t_repeat < 0 should generate all false.
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, i, (t_repeat < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_repeat < 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc_repeat
+        order by t_repeat, si, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: -- t_repeat < 0 should generate all false.
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, i, (t_repeat < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_repeat < 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc_repeat
+        order by t_repeat, si, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+-53707879050
+PREHOOK: query: -- Two repeated false columns at beginning...
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si_repeat, i, (t_repeat > 0) as child1, (si_repeat > 0) as child2, (i < 0) as child3, (t_repeat > 0 AND si_repeat > 0 AND i < 0) as multi_and_col from vectortab2k_orc_repeat
+        order by t_repeat, si_repeat, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: -- Two repeated false columns at beginning...
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si_repeat, i, (t_repeat > 0) as child1, (si_repeat > 0) as child2, (i < 0) as child3, (t_repeat > 0 AND si_repeat > 0 AND i < 0) as multi_and_col from vectortab2k_orc_repeat
+        order by t_repeat, si_repeat, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+-55121400269
+PREHOOK: query: -- si_repeat > 0 should generate all true.
+SELECT sum(hash(*)) FROM
+    (SELECT t, si_repeat, i, b_repeat, (t < 0 ) as child1, (si_repeat > 0) as child2, (i < 0) as child3, (b_repeat > 0) as child4, (t < 0 AND si_repeat > 0 AND i < 0 AND b_repeat > 0) as multi_and_col from vectortab2k_orc_repeat
+        order by t, si_repeat, i, b_repeat) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: -- si_repeat > 0 should generate all true.
+SELECT sum(hash(*)) FROM
+    (SELECT t, si_repeat, i, b_repeat, (t < 0 ) as child1, (si_repeat > 0) as child2, (i < 0) as child3, (b_repeat > 0) as child4, (t < 0 AND si_repeat > 0 AND i < 0 AND b_repeat > 0) as multi_and_col from vectortab2k_orc_repeat
+        order by t, si_repeat, i, b_repeat) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+-39803959564
+PREHOOK: query: -- si_repeat < 0 should generate all false.
+SELECT sum(hash(*)) FROM
+    (SELECT t, si_repeat, i, b_repeat, (t < 0) as child1, (si_repeat < 0) as child2, (i < 0) as child3, (b_repeat > 0) as child4, (t < 0 AND si_repeat < 0 AND i < 0 AND b_repeat > 0) as multi_and_col from vectortab2k_orc_repeat
+        order by t, si_repeat, i, b_repeat) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: -- si_repeat < 0 should generate all false.
+SELECT sum(hash(*)) FROM
+    (SELECT t, si_repeat, i, b_repeat, (t < 0) as child1, (si_repeat < 0) as child2, (i < 0) as child3, (b_repeat > 0) as child4, (t < 0 AND si_repeat < 0 AND i < 0 AND b_repeat > 0) as multi_and_col from vectortab2k_orc_repeat
+        order by t, si_repeat, i, b_repeat) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+-39863541564
+PREHOOK: query: -- Use a boolean column rather than a column comparison expression.
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo_repeat_false, b, (t < 0) as child1, (si > 0) as child2, (bo_repeat_false) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND bo_repeat_false AND b > 0) as multi_and_col from vectortab2k_orc_repeat
+        order by t, si, bo_repeat_false, b) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: -- Use a boolean column rather than a column comparison expression.
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo_repeat_false, b, (t < 0) as child1, (si > 0) as child2, (bo_repeat_false) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND bo_repeat_false AND b > 0) as multi_and_col from vectortab2k_orc_repeat
+        order by t, si, bo_repeat_false, b) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+56950496526
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo_repeat_true, b, (t < 0) as child1, (si > 0) as child2, (bo_repeat_true) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND bo_repeat_true AND b > 0) as multi_and_col from vectortab2k_orc_repeat
+        order by t, si, bo_repeat_true, b) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo_repeat_true, b, (t < 0) as child1, (si > 0) as child2, (bo_repeat_true) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND bo_repeat_true AND b > 0) as multi_and_col from vectortab2k_orc_repeat
+        order by t, si, bo_repeat_true, b) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+2303712725
+PREHOOK: query: -- With some filtering
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, i, (t_repeat > 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_repeat > 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc_repeat
+        where pmod(si, 4) = 0
+        order by t_repeat, si, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: -- With some filtering
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, i, (t_repeat > 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_repeat > 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc_repeat
+        where pmod(si, 4) = 0
+        order by t_repeat, si, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+-4381273667
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, i, (t_repeat < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_repeat < 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc_repeat
+        where pmod(si, 4) = 3
+        order by t_repeat, si, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, i, (t_repeat < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_repeat < 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc_repeat
+        where pmod(si, 4) = 3
+        order by t_repeat, si, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+-954303006
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si_repeat, i, (t_repeat > 0) as child1, (si_repeat > 0) as child2, (i < 0) as child3, (t_repeat > 0 AND si_repeat > 0 AND i < 0) as multi_and_col from vectortab2k_orc_repeat
+        where pmod(si, 4) != 3
+        order by t_repeat, si_repeat, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si_repeat, i, (t_repeat > 0) as child1, (si_repeat > 0) as child2, (i < 0) as child3, (t_repeat > 0 AND si_repeat > 0 AND i < 0) as multi_and_col from vectortab2k_orc_repeat
+        where pmod(si, 4) != 3
+        order by t_repeat, si_repeat, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+-45312591578
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si_repeat, i, b_repeat, (t < 0 ) as child1, (si_repeat > 0) as child2, (i < 0) as child3, (b_repeat > 0) as child4, (t < 0 AND si_repeat > 0 AND i < 0 AND b_repeat > 0) as multi_and_col from vectortab2k_orc_repeat
+        where pmod(si, 4) < 2
+        order by t, si_repeat, i, b_repeat) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si_repeat, i, b_repeat, (t < 0 ) as child1, (si_repeat > 0) as child2, (i < 0) as child3, (b_repeat > 0) as child4, (t < 0 AND si_repeat > 0 AND i < 0 AND b_repeat > 0) as multi_and_col from vectortab2k_orc_repeat
+        where pmod(si, 4) < 2
+        order by t, si_repeat, i, b_repeat) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+-30297068525
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si_repeat, i, b_repeat, (t < 0) as child1, (si_repeat < 0) as child2, (i < 0) as child3, (b_repeat > 0) as child4, (t < 0 AND si_repeat < 0 AND i < 0 AND b_repeat > 0) as multi_and_col from vectortab2k_orc_repeat
+        where pmod(t, 4) = 0
+        order by t, si_repeat, i, b_repeat) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si_repeat, i, b_repeat, (t < 0) as child1, (si_repeat < 0) as child2, (i < 0) as child3, (b_repeat > 0) as child4, (t < 0 AND si_repeat < 0 AND i < 0 AND b_repeat > 0) as multi_and_col from vectortab2k_orc_repeat
+        where pmod(t, 4) = 0
+        order by t, si_repeat, i, b_repeat) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+32929905326
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo_repeat_false, b, (t < 0) as child1, (si > 0) as child2, (bo_repeat_false) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND bo_repeat_false AND b > 0) as multi_and_col from vectortab2k_orc_repeat
+        where pmod(b, 4) > 1
+        order by t, si, bo_repeat_false, b) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo_repeat_false, b, (t < 0) as child1, (si > 0) as child2, (bo_repeat_false) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND bo_repeat_false AND b > 0) as multi_and_col from vectortab2k_orc_repeat
+        where pmod(b, 4) > 1
+        order by t, si, bo_repeat_false, b) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+10610127509
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo_repeat_true, b, (t < 0) as child1, (si > 0) as child2, (bo_repeat_true) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND bo_repeat_true AND b > 0) as multi_and_col from vectortab2k_orc_repeat
+        where pmod(si, 4) < 3
+        order by t, si, bo_repeat_true, b) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo_repeat_true, b, (t < 0) as child1, (si > 0) as child2, (bo_repeat_true) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND bo_repeat_true AND b > 0) as multi_and_col from vectortab2k_orc_repeat
+        where pmod(si, 4) < 3
+        order by t, si, bo_repeat_true, b) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+-16740409601
+PREHOOK: query: CREATE TABLE scratch_null AS SELECT t, si, i, b, bo,
+     cast(null as tinyint) as t_null, cast(null as smallint) as si_null, cast(null as int) as i_null, cast(null as bigint) as b_null, cast(null as boolean) as bo_null FROM vectortab2k
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@vectortab2k
+PREHOOK: Output: database:default
+PREHOOK: Output: default@scratch_null
+POSTHOOK: query: CREATE TABLE scratch_null AS SELECT t, si, i, b, bo,
+     cast(null as tinyint) as t_null, cast(null as smallint) as si_null, cast(null as int) as i_null, cast(null as bigint) as b_null, cast(null as boolean) as bo_null FROM vectortab2k
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@vectortab2k
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@scratch_null
+POSTHOOK: Lineage: scratch_null.b SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: scratch_null.b_null SIMPLE []
+POSTHOOK: Lineage: scratch_null.bo SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: scratch_null.bo_null SIMPLE []
+POSTHOOK: Lineage: scratch_null.i SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: scratch_null.i_null SIMPLE []
+POSTHOOK: Lineage: scratch_null.si SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: scratch_null.si_null SIMPLE []
+POSTHOOK: Lineage: scratch_null.t SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:t, type:tinyint, comment:null), ]
+POSTHOOK: Lineage: scratch_null.t_null SIMPLE []
+t	si	i	b	bo	t_null	si_null	i_null	b_null	bo_null
+PREHOOK: query: -- The nulled columns ought to create repeated null VectorizedRowBatch for those columns.
+CREATE TABLE vectortab2k_orc_null STORED AS ORC AS SELECT * FROM scratch_null
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@scratch_null
+PREHOOK: Output: database:default
+PREHOOK: Output: default@vectortab2k_orc_null
+POSTHOOK: query: -- The nulled columns ought to create repeated null VectorizedRowBatch for those columns.
+CREATE TABLE vectortab2k_orc_null STORED AS ORC AS SELECT * FROM scratch_null
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@scratch_null
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@vectortab2k_orc_null
+POSTHOOK: Lineage: vectortab2k_orc_null.b SIMPLE [(scratch_null)scratch_null.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_null.b_null SIMPLE [(scratch_null)scratch_null.FieldSchema(name:b_null, type:bigint, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_null.bo SIMPLE [(scratch_null)scratch_null.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_null.bo_null SIMPLE [(scratch_null)scratch_null.FieldSchema(name:bo_null, type:boolean, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_null.i SIMPLE [(scratch_null)scratch_null.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_null.i_null SIMPLE [(scratch_null)scratch_null.FieldSchema(name:i_null, type:int, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_null.si SIMPLE [(scratch_null)scratch_null.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_null.si_null SIMPLE [(scratch_null)scratch_null.FieldSchema(name:si_null, type:smallint, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_null.t SIMPLE [(scratch_null)scratch_null.FieldSchema(name:t, type:tinyint, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_null.t_null SIMPLE [(scratch_null)scratch_null.FieldSchema(name:t_null, type:tinyint, comment:null), ]
+scratch_null.t	scratch_null.si	scratch_null.i	scratch_null.b	scratch_null.bo	scratch_null.t_null	scratch_null.si_null	scratch_null.i_null	scratch_null.b_null	scratch_null.bo_null
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_null, si, i, (t_null is null) as child1, (si > 0) as child2, (i < 0) as child3, (t_null is null AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc_null
+        order by t_null, si, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_null, si, i, (t_null is null) as child1, (si > 0) as child2, (i < 0) as child3, (t_null is null AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc_null
+        order by t_null, si, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+c0
+-77365671658
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_null, si, i, (t_null < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_null < 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc_null
+        order by t_null, si, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_null, si, i, (t_null < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_null < 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc_null
+        order by t_null, si, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+c0
+-77425254090
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_null, si_null, i, (t_null is null) as child1, (si_null > 0) as child2, (i < 0) as child3, (t_null is null AND si_null > 0 AND i < 0) as multi_and_col from vectortab2k_orc_null
+        order by t_null, si_null, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_null, si_null, i, (t_null is null) as child1, (si_null > 0) as child2, (i < 0) as child3, (t_null is null AND si_null > 0 AND i < 0) as multi_and_col from vectortab2k_orc_null
+        order by t_null, si_null, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+c0
+-127742262531
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_null, si_null, i_null, (t_null is null) as child1, (si_null is null) as child2, (i_null is null) as child3, (t_null is null AND si_null is null AND i_null is null) as multi_and_col from vectortab2k_orc_null
+        order by t_null, si_null, i_null) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_null, si_null, i_null, (t_null is null) as child1, (si_null is null) as child2, (i_null is null) as child3, (t_null is null AND si_null is null AND i_null is null) as multi_and_col from vectortab2k_orc_null
+        order by t_null, si_null, i_null) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+c0
+61568000
+PREHOOK: query: -- With some filtering
+SELECT sum(hash(*)) FROM
+    (SELECT t_null, si, i, (t_null is null) as child1, (si > 0) as child2, (i < 0) as child3, (t_null is null AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc_null
+        where pmod(i,4) = 3
+        order by t_null, si, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+POSTHOOK: query: -- With some filtering
+SELECT sum(hash(*)) FROM
+    (SELECT t_null, si, i, (t_null is null) as child1, (si > 0) as child2, (i < 0) as child3, (t_null is null AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc_null
+        where pmod(i,4) = 3
+        order by t_null, si, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+c0
+-48771559290
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_null, si, i, (t_null < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_null < 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc_null
+        where pmod(i,4) = 2
+        order by t_null, si, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_null, si, i, (t_null < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_null < 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc_null
+        where pmod(i,4) = 2
+        order by t_null, si, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+c0
+-13491514290
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_null, si_null, i, (t_null is null) as child1, (si_null > 0) as child2, (i < 0) as child3, (t_null is null AND si_null > 0 AND i < 0) as multi_and_col from vectortab2k_orc_null
+        where pmod(i,4) != 3
+        order by t_null, si_null, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_null, si_null, i, (t_null is null) as child1, (si_null > 0) as child2, (i < 0) as child3, (t_null is null AND si_null > 0 AND i < 0) as multi_and_col from vectortab2k_orc_null
+        where pmod(i,4) != 3
+        order by t_null, si_null, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+c0
+-96004829404
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_null, si_null, i_null, (t_null is null) as child1, (si_null is null) as child2, (i_null is null) as child3, (t_null is null AND si_null is null AND i_null is null) as multi_and_col from vectortab2k_orc_null
+        where pmod(i,4) < 3
+        order by t_null, si_null, i_null) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_null, si_null, i_null, (t_null is null) as child1, (si_null is null) as child2, (i_null is null) as child3, (t_null is null AND si_null is null AND i_null is null) as multi_and_col from vectortab2k_orc_null
+        where pmod(i,4) < 3
+        order by t_null, si_null, i_null) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+c0
+45252480


[5/5] hive git commit: HIVE-13084: Vectorization add support for PROJECTION Multi-AND/OR (Matt McCline, reviewed by Sergey Shelukhin)

Posted by mm...@apache.org.
HIVE-13084: Vectorization add support for PROJECTION Multi-AND/OR (Matt McCline, reviewed by Sergey Shelukhin)


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

Branch: refs/heads/branch-2.1
Commit: 9dd70f0d38674370a557465f0acec5e1cfa5e8d1
Parents: 2bd2e9a
Author: Matt McCline <mm...@hortonworks.com>
Authored: Fri May 27 20:00:17 2016 -0700
Committer: Matt McCline <mm...@hortonworks.com>
Committed: Fri May 27 20:29:43 2016 -0700

----------------------------------------------------------------------
 .../vectorization/VectorizedLogicBench.java     |  12 +-
 .../test/resources/testconfiguration.properties |   2 +
 .../ql/exec/vector/VectorizationContext.java    | 125 ++-
 .../ql/exec/vector/expressions/ColAndCol.java   | 687 +++++++++++-----
 .../ql/exec/vector/expressions/ColOrCol.java    | 694 +++++++++++-----
 .../exec/vector/TestVectorizationContext.java   |   4 -
 .../TestVectorLogicalExpressions.java           |   4 +-
 .../vector_multi_and_projection.q               | 196 +++++
 .../clientpositive/vector_multi_or_projection.q | 198 +++++
 .../tez/vector_multi_and_projection.q.out       | 821 +++++++++++++++++++
 .../tez/vector_multi_or_projection.q.out        | 821 +++++++++++++++++++
 .../vector_multi_and_projection.q.out           | 800 ++++++++++++++++++
 .../vector_multi_or_projection.q.out            | 800 ++++++++++++++++++
 13 files changed, 4667 insertions(+), 497 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/9dd70f0d/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/VectorizedLogicBench.java
----------------------------------------------------------------------
diff --git a/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/VectorizedLogicBench.java b/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/VectorizedLogicBench.java
index 50dadb2..7ff6158 100644
--- a/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/VectorizedLogicBench.java
+++ b/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/VectorizedLogicBench.java
@@ -55,7 +55,7 @@ public class VectorizedLogicBench {
     public void setup() {
       rowBatch = buildRowBatch(new LongColumnVector(), 2, getBooleanLongColumnVector(),
           getBooleanLongColumnVector());
-      expression = new ColAndCol(0, 1, 2);
+      expression = new ColAndCol(new int[] {0, 1}, 2);
     }
   }
 
@@ -64,7 +64,7 @@ public class VectorizedLogicBench {
     public void setup() {
       rowBatch = buildRowBatch(new LongColumnVector(), 2, getBooleanLongColumnVector(),
           getBooleanRepeatingLongColumnVector());
-      expression = new ColAndCol(0, 1, 2);
+      expression = new ColAndCol(new int[] {0, 1}, 2);
     }
   }
 
@@ -73,7 +73,7 @@ public class VectorizedLogicBench {
     public void setup() {
       rowBatch = buildRowBatch(new LongColumnVector(), 2, getBooleanRepeatingLongColumnVector(),
           getBooleanLongColumnVector());
-      expression = new ColAndCol(0, 1, 2);
+      expression = new ColAndCol(new int[] {0, 1}, 2);
     }
   }
 
@@ -82,7 +82,7 @@ public class VectorizedLogicBench {
     public void setup() {
       rowBatch = buildRowBatch(new LongColumnVector(), 2, getBooleanLongColumnVector(),
           getBooleanLongColumnVector());
-      expression = new ColOrCol(0, 1, 2);
+      expression = new ColOrCol(new int[] {0, 1}, 2);
     }
   }
 
@@ -91,7 +91,7 @@ public class VectorizedLogicBench {
     public void setup() {
       rowBatch = buildRowBatch(new LongColumnVector(), 2, getBooleanLongColumnVector(),
           getBooleanRepeatingLongColumnVector());
-      expression = new ColOrCol(0, 1, 2);
+      expression = new ColOrCol(new int[] {0, 1}, 2);
     }
   }
 
@@ -100,7 +100,7 @@ public class VectorizedLogicBench {
     public void setup() {
       rowBatch = buildRowBatch(new LongColumnVector(), 2, getBooleanRepeatingLongColumnVector(),
           getBooleanLongColumnVector());
-      expression = new ColOrCol(0, 1, 2);
+      expression = new ColOrCol(new int[] {0, 1}, 2);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/9dd70f0d/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index 1ab914d..fd6901c 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -311,7 +311,9 @@ minitez.query.files.shared=acid_globallimit.q,\
   vector_leftsemi_mapjoin.q,\
   vector_mapjoin_reduce.q,\
   vector_mr_diff_schema_alias.q,\
+  vector_multi_and_projection.q,\
   vector_multi_insert.q,\
+  vector_multi_or_projection.q,\
   vector_non_string_partition.q,\
   vector_nullsafe_join.q,\
   vector_null_projection.q,\

http://git-wip-us.apache.org/repos/asf/hive/blob/9dd70f0d/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 7f55b31..58ce063 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
@@ -122,6 +122,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
 import org.apache.hadoop.hive.serde2.typeinfo.BaseCharTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
@@ -965,36 +966,33 @@ public class VectorizationContext {
 
     int numChildren = (childExpr == null) ? 0 : childExpr.size();
 
-    if (numChildren > 2 && genericeUdf != null && mode == Mode.FILTER &&
+    if (genericeUdf != null &&
         ((genericeUdf instanceof GenericUDFOPOr) || (genericeUdf instanceof GenericUDFOPAnd))) {
 
-      // Special case handling for Multi-OR and Multi-AND.
+      // Special case handling for Multi-OR and Multi-AND FILTER and PROJECTION.
 
-      for (int i = 0; i < numChildren; i++) {
-        ExprNodeDesc child = childExpr.get(i);
-        String childTypeString = child.getTypeString();
-        if (childTypeString == null) {
-          throw new HiveException("Null child type name string");
-        }
-        TypeInfo typeInfo = TypeInfoUtils.getTypeInfoFromTypeString(childTypeString);
-        Type columnVectorType = VectorizationContext.getColumnVectorTypeFromTypeInfo(typeInfo);
-        if (columnVectorType != ColumnVector.Type.LONG){
-          return null;
-        }
-        if (!(child instanceof ExprNodeGenericFuncDesc) && !(child instanceof ExprNodeColumnDesc)) {
-          return null;
-        }
-      }
       Class<?> vclass;
       if (genericeUdf instanceof GenericUDFOPOr) {
-        vclass = FilterExprOrExpr.class;
+        if (mode == Mode.PROJECTION) {
+          vclass = ColOrCol.class;
+        } else {
+          vclass = FilterExprOrExpr.class;
+        }
       } else if (genericeUdf instanceof GenericUDFOPAnd) {
-        vclass = FilterExprAndExpr.class;
+        if (mode == Mode.PROJECTION) {
+          vclass = ColAndCol.class;
+        } else {
+          vclass = FilterExprAndExpr.class;
+        }
       } else {
         throw new RuntimeException("Unexpected multi-child UDF");
       }
       Mode childrenMode = getChildrenMode(mode, udfClass);
-      return createVectorExpression(vclass, childExpr, childrenMode, returnType);
+      if (mode == Mode.PROJECTION) {
+        return createVectorMultiAndOrProjectionExpr(vclass, childExpr, childrenMode, returnType);
+      } else {
+        return createVectorExpression(vclass, childExpr, childrenMode, returnType);
+      }
     }
     if (numChildren > VectorExpressionDescriptor.MAX_NUM_ARGUMENTS) {
       return null;
@@ -1033,6 +1031,37 @@ public class VectorizationContext {
     return createVectorExpression(vclass, childExpr, childrenMode, returnType);
   }
 
+  private void determineChildrenVectorExprAndArguments(Class<?> vectorClass,
+      List<ExprNodeDesc> childExpr, int numChildren, Mode childrenMode,
+      VectorExpression.Type [] inputTypes, List<VectorExpression> children, Object[] arguments)
+          throws HiveException {
+    for (int i = 0; i < numChildren; i++) {
+      ExprNodeDesc child = childExpr.get(i);
+      String undecoratedName = getUndecoratedName(child.getTypeInfo().getTypeName());
+      inputTypes[i] = VectorExpression.Type.getValue(undecoratedName);
+      if (inputTypes[i] == VectorExpression.Type.OTHER){
+        throw new HiveException("No vector type for " + vectorClass.getSimpleName() + " argument #" + i + " type name " + undecoratedName);
+      }
+      if (child instanceof ExprNodeGenericFuncDesc) {
+        VectorExpression vChild = getVectorExpression(child, childrenMode);
+          children.add(vChild);
+          arguments[i] = vChild.getOutputColumn();
+      } else if (child instanceof ExprNodeColumnDesc) {
+        int colIndex = getInputColumnIndex((ExprNodeColumnDesc) child);
+          if (childrenMode == Mode.FILTER) {
+            // In filter mode, the column must be a boolean
+            children.add(new SelectColumnIsTrue(colIndex));
+          }
+          arguments[i] = colIndex;
+      } else if (child instanceof ExprNodeConstantDesc) {
+        Object scalarValue = getVectorTypeScalarValue((ExprNodeConstantDesc) child);
+        arguments[i] = (null == scalarValue) ? getConstantVectorExpression(null, child.getTypeInfo(), childrenMode) : scalarValue;
+      } else {
+        throw new HiveException("Cannot handle expression type: " + child.getClass().getSimpleName());
+      }
+    }
+  }
+
   private VectorExpression createVectorExpression(Class<?> vectorClass,
       List<ExprNodeDesc> childExpr, Mode childrenMode, TypeInfo returnType) throws HiveException {
     int numChildren = childExpr == null ? 0: childExpr.size();
@@ -1040,31 +1069,41 @@ public class VectorizationContext {
     List<VectorExpression> children = new ArrayList<VectorExpression>();
     Object[] arguments = new Object[numChildren];
     try {
+      determineChildrenVectorExprAndArguments(vectorClass, childExpr, numChildren, childrenMode,
+          inputTypes, children, arguments);
+      VectorExpression  vectorExpression = instantiateExpression(vectorClass, returnType, arguments);
+      vectorExpression.setInputTypes(inputTypes);
+      if ((vectorExpression != null) && !children.isEmpty()) {
+        vectorExpression.setChildExpressions(children.toArray(new VectorExpression[0]));
+      }
+      return vectorExpression;
+    } catch (Exception ex) {
+      throw new HiveException(ex);
+    } finally {
+      for (VectorExpression ve : children) {
+        ocm.freeOutputColumn(ve.getOutputColumn());
+      }
+    }
+  }
+
+  private VectorExpression createVectorMultiAndOrProjectionExpr(Class<?> vectorClass,
+      List<ExprNodeDesc> childExpr, Mode childrenMode, TypeInfo returnType) throws HiveException {
+    int numChildren = childExpr == null ? 0: childExpr.size();
+    VectorExpression.Type [] inputTypes = new VectorExpression.Type[numChildren];
+    List<VectorExpression> children = new ArrayList<VectorExpression>();
+    Object[] arguments = new Object[numChildren];
+    try {
+      determineChildrenVectorExprAndArguments(vectorClass, childExpr, numChildren, childrenMode,
+          inputTypes, children, arguments);
+
+      // For Multi-AND/OR, transform the arguments -- column indices into an array of int.
+      int[] colNums = new int[numChildren];
       for (int i = 0; i < numChildren; i++) {
-        ExprNodeDesc child = childExpr.get(i);
-        String undecoratedName = getUndecoratedName(child.getTypeInfo().getTypeName());
-        inputTypes[i] = VectorExpression.Type.getValue(undecoratedName);
-        if (inputTypes[i] == VectorExpression.Type.OTHER){
-          throw new HiveException("No vector type for " + vectorClass.getSimpleName() + " argument #" + i + " type name " + undecoratedName);
-        }
-        if (child instanceof ExprNodeGenericFuncDesc) {
-          VectorExpression vChild = getVectorExpression(child, childrenMode);
-            children.add(vChild);
-            arguments[i] = vChild.getOutputColumn();
-        } else if (child instanceof ExprNodeColumnDesc) {
-          int colIndex = getInputColumnIndex((ExprNodeColumnDesc) child);
-            if (childrenMode == Mode.FILTER) {
-              // In filter mode, the column must be a boolean
-              children.add(new SelectColumnIsTrue(colIndex));
-            }
-            arguments[i] = colIndex;
-        } else if (child instanceof ExprNodeConstantDesc) {
-          Object scalarValue = getVectorTypeScalarValue((ExprNodeConstantDesc) child);
-          arguments[i] = (null == scalarValue) ? getConstantVectorExpression(null, child.getTypeInfo(), childrenMode) : scalarValue;
-        } else {
-          throw new HiveException("Cannot handle expression type: " + child.getClass().getSimpleName());
-        }
+        colNums[i] = (Integer) arguments[i];
       }
+      arguments = new Object[1];
+      arguments[0] = colNums;
+
       VectorExpression  vectorExpression = instantiateExpression(vectorClass, returnType, arguments);
       vectorExpression.setInputTypes(inputTypes);
       if ((vectorExpression != null) && !children.isEmpty()) {

http://git-wip-us.apache.org/repos/asf/hive/blob/9dd70f0d/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ColAndCol.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ColAndCol.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ColAndCol.java
index ff7371d..48e3070 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ColAndCol.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ColAndCol.java
@@ -17,26 +17,38 @@
  */
 package org.apache.hadoop.hive.ql.exec.vector.expressions;
 
+import java.util.Arrays;
+
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 
+import com.google.common.base.Preconditions;
+
 /**
- * Evaluate AND of two boolean columns and store result in the output boolean column.
+ * Evaluate AND of 2 or more boolean columns and store the boolean result in the
+ * output boolean column.  This is a projection or result producing expression (as opposed to
+ * a filter expression).
+ *
+ * Some child boolean columns may be vector expressions evaluated into boolean scratch columns.
  */
 public class ColAndCol extends VectorExpression {
 
   private static final long serialVersionUID = 1L;
 
-  private int colNum1;
-  private int colNum2;
+  private int[] colNums;
   private int outputColumn;
+  private int[] mapToChildExpression;
+  private int[] andSelected;
+  private boolean[] intermediateNulls;
 
-  public ColAndCol(int colNum1, int colNum2, int outputColumn) {
+  public ColAndCol(int[] colNums, int outputColumn) {
     this();
-    this.colNum1 = colNum1;
-    this.colNum2 = colNum2;
+    this.colNums = colNums;
     this.outputColumn = outputColumn;
+    mapToChildExpression = null;
+    andSelected = new int[VectorizedRowBatch.DEFAULT_SIZE];
+    intermediateNulls = new boolean[VectorizedRowBatch.DEFAULT_SIZE];
   }
 
   public ColAndCol() {
@@ -46,240 +58,493 @@ public class ColAndCol extends VectorExpression {
   @Override
   public void evaluate(VectorizedRowBatch batch) {
 
+    Preconditions.checkState(colNums.length >= 2);
+
+    /*
+     * Vector child expressions will be omitted if they are existing boolean vector columns,
+     * so the child index does not necessarily index into the childExpressions.
+     * We construct a simple index map to the child expression in mapToChildExpression.
+     */
+    if (childExpressions != null && mapToChildExpression == null) {
+      // 
+      mapToChildExpression = new int [colNums.length];
+      int childIndex = 0;
+      for (int i = 0; i < childExpressions.length; i++) {
+        VectorExpression ve = childExpressions[i];
+        int outputColumn = ve.getOutputColumn();
+        while (outputColumn != colNums[childIndex]) {
+          mapToChildExpression[childIndex++] = -1;
+        }
+        mapToChildExpression[childIndex++] = i;
+      }
+      Preconditions.checkState(childIndex == colNums.length);
+    }
+
+    final int n = batch.size;
+    if (n <= 0) {
+      // Nothing to do
+      return;
+    }
+
     if (childExpressions != null) {
-      super.evaluateChildren(batch);
+      /*
+       * Evaluate first child expression.  Other child are conditionally evaluated later
+       * based on whether there is still a need for AND processing.
+       */
+      int childExpressionIndex = mapToChildExpression[0];
+      if (childExpressionIndex != -1) {
+        VectorExpression ve = childExpressions[childExpressionIndex];
+        Preconditions.checkState(ve.getOutputColumn() == colNums[0]);
+        ve.evaluate(batch);
+      }
     }
 
-    LongColumnVector inputColVector1 = (LongColumnVector) batch.cols[colNum1];
-    LongColumnVector inputColVector2 = (LongColumnVector) batch.cols[colNum2];
     int[] sel = batch.selected;
-    int n = batch.size;
-    long[] vector1 = inputColVector1.vector;
-    long[] vector2 = inputColVector2.vector;
 
     LongColumnVector outV = (LongColumnVector) batch.cols[outputColumn];
     long[] outputVector = outV.vector;
-    if (n <= 0) {
-      // Nothing to do
-      return;
-    }
 
-    long vector1Value = vector1[0];
-    long vector2Value = vector2[0];
-    if (inputColVector1.noNulls && inputColVector2.noNulls) {
-      if ((inputColVector1.isRepeating) && (inputColVector2.isRepeating)) {
-        // All must be selected otherwise size would be zero
-        // Repeating property will not change.
-        outV.isRepeating = true;
-        outputVector[0] = vector1[0] & vector2[0];
-      } else if (inputColVector1.isRepeating && !inputColVector2.isRepeating) {
-        if (batch.selectedInUse) {
-          for (int j = 0; j != n; j++) {
-            int i = sel[j];
-            outputVector[i] = vector1Value & vector2[i];
-          }
+    /**
+     * Null processing complicates the algorithm here for Multi-AND.
+     *
+     * All true --> true
+     * 0 or more true with 1 or more null --> result = null
+     * Any false --> false
+     *
+     * For AND-processing, we remember nulls in the intermediateNulls array as we go along so
+     * later we can mark the row as null instead of true when there is a null.
+     */
+
+    /*
+     * andRepeating will be true when all the children column vectors processed so far are
+     * some combination of repeating true and repeating null.
+     * andRepeatingIsNull will be true when there has been at least one repeating null column.
+     */
+    boolean andRepeating = false;
+    boolean andRepeatingIsNull = false;
+
+    /*
+     * The andSel variable and andSelected member array represent rows that have at have
+     * some combination of true and nulls.
+     */
+    int andSel = 0;
+
+    Arrays.fill(intermediateNulls, 0, VectorizedRowBatch.DEFAULT_SIZE, false);
+
+    // Reset noNulls to true, isNull to false, isRepeating to false.
+    outV.reset();
+
+    LongColumnVector firstColVector = (LongColumnVector) batch.cols[colNums[0]];
+    long[] firstVector = firstColVector.vector;
+
+    /*
+     * We prime the pump by evaluating the first child to see if we are starting with
+     * andRepeating/andRepeatingHasNulls or we are starting with andSel/andSelected processing.
+     */
+    if (firstColVector.isRepeating) {
+      if (firstColVector.noNulls || !firstColVector.isNull[0]) {
+        if (firstVector[0] == 0) {
+          // When the entire child column is repeating false, we are done for AND.
+          outV.isRepeating = true;
+          outputVector[0] = 0;
+          return;
         } else {
-          for (int i = 0; i != n; i++) {
-            outputVector[i] = vector1Value & vector2[i];
-          }
+          // First column is repeating true.
         }
-        outV.isRepeating = false;
-      } else if (!inputColVector1.isRepeating && inputColVector2.isRepeating) {
-        if (batch.selectedInUse) {
-          for (int j = 0; j != n; j++) {
-            int i = sel[j];
-            outputVector[i] = vector1[i] & vector2Value;
-          }
-        } else {
-          for (int i = 0; i != n; i++) {
-            outputVector[i] = vector1[i] & vector2Value;
+      } else {
+        Preconditions.checkState(firstColVector.isNull[0]);
+
+        // At least one repeating null column.
+        andRepeatingIsNull = true;
+      }
+      andRepeating = true;
+    } else if (firstColVector.noNulls) {
+
+      /*
+       * No nulls -- so all true rows go in andSel/andSelected.
+       */
+      if (batch.selectedInUse) {
+        for (int j = 0; j != n; j++) {
+          int i = sel[j];
+          if (firstVector[i] == 1) {
+            andSelected[andSel++] = i;
           }
         }
-        outV.isRepeating = false;
-      } else /* neither side is repeating */{
-        if (batch.selectedInUse) {
-          for (int j = 0; j != n; j++) {
-            int i = sel[j];
-            outputVector[i] = vector1[i] & vector2[i];
-          }
-        } else {
-          for (int i = 0; i != n; i++) {
-            outputVector[i] = vector1[i] & vector2[i];
+      } else {
+        for (int i = 0; i != n; i++) {
+          if (firstVector[i] == 1) {
+            andSelected[andSel++] = i;
           }
         }
-        outV.isRepeating = false;
       }
-      outV.noNulls = true;
-    } else if (inputColVector1.noNulls && !inputColVector2.noNulls) {
-      // only input 2 side has nulls
-      if ((inputColVector1.isRepeating) && (inputColVector2.isRepeating)) {
-        // All must be selected otherwise size would be zero
-        // Repeating property will not change.
-        outV.isRepeating = true;
-        outputVector[0] = vector1[0] & vector2[0];
-        outV.isNull[0] = (vector1[0] == 1) && inputColVector2.isNull[0];
-      } else if (inputColVector1.isRepeating && !inputColVector2.isRepeating) {
-        if (batch.selectedInUse) {
-          for (int j = 0; j != n; j++) {
-            int i = sel[j];
-            outputVector[i] = vector1Value & vector2[i];
-            outV.isNull[i] = (vector1[0] == 1) && inputColVector2.isNull[i];
-          }
-        } else {
-          for (int i = 0; i != n; i++) {
-            outputVector[i] = vector1Value & vector2[i];
-            outV.isNull[i] = (vector1[0] == 1) && inputColVector2.isNull[i];
+    } else  {
+
+      /*
+       * Can be nulls -- so all true rows and null rows go in andSel/andSelected.
+       * Remember nulls in our separate intermediateNulls array.
+       */
+      if (batch.selectedInUse) {
+        for (int j = 0; j != n; j++) {
+          int i = sel[j];
+          if (firstColVector.isNull[i]) {
+            intermediateNulls[i] = true;
+            andSelected[andSel++] = i;
+          } else if (firstVector[i] == 1) {
+            andSelected[andSel++] = i;
           }
         }
-        outV.isRepeating = false;
-      } else if (!inputColVector1.isRepeating && inputColVector2.isRepeating) {
-        if (batch.selectedInUse) {
-          for (int j = 0; j != n; j++) {
-            int i = sel[j];
-            outputVector[i] = vector1[i] & vector2Value;
-            outV.isNull[i] = (vector1[i] == 1) && inputColVector2.isNull[0];
-          }
-        } else {
-          for (int i = 0; i != n; i++) {
-            outputVector[i] = vector1[i] & vector2Value;
-            outV.isNull[i] = (vector1[i] == 1) && inputColVector2.isNull[0];
+      } else {
+        for (int i = 0; i != n; i++) {
+          if (firstColVector.isNull[i]) {
+            intermediateNulls[i] = true;
+            andSelected[andSel++] = i;
+          } else if (firstVector[i] == 1) {
+            andSelected[andSel++] = i;
           }
         }
-        outV.isRepeating = false;
-      } else /* neither side is repeating */{
-        if (batch.selectedInUse) {
-          for (int j = 0; j != n; j++) {
-            int i = sel[j];
-            outputVector[i] = vector1[i] & vector2[i];
-            outV.isNull[i] = (vector1[i] == 1) && inputColVector2.isNull[i];
-          }
-        } else {
-          for (int i = 0; i != n; i++) {
-            outputVector[i] = vector1[i] & vector2[i];
-            outV.isNull[i] = (vector1[i] == 1) && inputColVector2.isNull[i];
+      }
+    }
+
+    /*
+     * Process child #2 and above.
+     */
+    int colNum = 1;
+    do {
+      if (!andRepeating && andSel == 0) {
+
+        /*
+         * Since andSel/andSelected represent currently true entries and there are none,
+         * then nothing is true (how philosophical!).
+         */
+        break;
+      }
+
+      if (childExpressions != null) {
+        int childExpressionIndex = mapToChildExpression[colNum];
+        if (childExpressionIndex != -1) {
+          if (andRepeating) {
+
+            /*
+             * We need to start with a full evaluate on all [selected] rows.
+             */
+            VectorExpression ve = childExpressions[childExpressionIndex];
+            Preconditions.checkState(ve.getOutputColumn() == colNums[colNum]);
+            ve.evaluate(batch);
+          } else {
+
+            /*
+             * Evaluate next child expression.
+             * But only evaluate the andSelected rows (i.e. current true or true with nulls rows).
+             */
+            boolean saveSelectedInUse = batch.selectedInUse;
+            int[] saveSelected = sel;
+            batch.selectedInUse = true;
+            batch.selected = andSelected;
+
+            VectorExpression ve = childExpressions[childExpressionIndex];
+            Preconditions.checkState(ve.getOutputColumn() == colNums[colNum]);
+            ve.evaluate(batch);
+
+            batch.selectedInUse = saveSelectedInUse;
+            batch.selected = saveSelected;
           }
         }
-        outV.isRepeating = false;
       }
-      outV.noNulls = false;
-    } else if (!inputColVector1.noNulls && inputColVector2.noNulls) {
-      // only input 1 side has nulls
-      if ((inputColVector1.isRepeating) && (inputColVector2.isRepeating)) {
-        // All must be selected otherwise size would be zero
-        // Repeating property will not change.
-        outV.isRepeating = true;
-        outputVector[0] = vector1[0] & vector2[0];
-        outV.isNull[0] = inputColVector1.isNull[0] && (vector2[0] == 1);
-      } else if (inputColVector1.isRepeating && !inputColVector2.isRepeating) {
-        if (batch.selectedInUse) {
-          for (int j = 0; j != n; j++) {
-            int i = sel[j];
-            outputVector[i] = vector1Value & vector2[i];
-            outV.isNull[i] = inputColVector1.isNull[0] && (vector2[i] == 1);
+
+      LongColumnVector nextColVector = (LongColumnVector) batch.cols[colNums[colNum]];
+      long[] nextVector = nextColVector.vector;
+
+      if (andRepeating) {
+
+        /*
+         * The andRepeating flag means the whole batch is repeating true possibly with
+         * some repeating nulls.
+         */
+        if (nextColVector.isRepeating) {
+
+          /*
+           * Current child column is repeating so stay in repeating mode.
+           */
+          if (nextColVector.noNulls || !nextColVector.isNull[0]) {
+            if (nextVector[0] == 0) {
+              // When the entire child column is repeating false, we are done for AND.
+              outV.isRepeating = true;
+              outputVector[0] = 0;
+              return;
+            } else {
+              // Current column is repeating true.
+            }
+          } else {
+            Preconditions.checkState(nextColVector.isNull[0]);
+
+            // At least one repeating null column.
+            andRepeatingIsNull = true;
           }
+          // Continue with andRepeating as true.
         } else {
-          for (int i = 0; i != n; i++) {
-            outputVector[i] = vector1Value & vector2[i];
-            outV.isNull[i] = inputColVector1.isNull[0] && (vector2[i] == 1);
+
+          /*
+           * Switch away from andRepeating/andRepeatingIsNull and now represent individual rows in
+           * andSel/andSelected.
+           */
+          if (nextColVector.noNulls) {
+
+            /*
+             * Current child column has no nulls.
+             */
+
+            Preconditions.checkState(andSel == 0);
+            andRepeating = false;
+  
+            if (andRepeatingIsNull) {
+  
+              /*
+               * Since andRepeatingIsNull is true, we always set intermediateNulls when building
+               * andSel/andSelected when the next row is true.
+               */
+              if (batch.selectedInUse) {
+                for (int j = 0; j != n; j++) {
+                  int i = sel[j];
+                  if (nextVector[i] == 1) {
+                    intermediateNulls[i] = true;
+                    andSelected[andSel++] = i;
+                  }
+                }
+              } else {
+                for (int i = 0; i != n; i++) {
+                  if (nextVector[i] == 1) {
+                    intermediateNulls[i] = true;
+                    andSelected[andSel++] = i;
+                  }
+                }
+              }
+              andRepeatingIsNull = false;
+            } else {
+  
+              /*
+               * Previous rounds were all true with no null child columns.  Just build
+               * andSel/andSelected when the next row is true.
+               */
+              if (batch.selectedInUse) {
+                for (int j = 0; j != n; j++) {
+                  int i = sel[j];
+                  if (nextVector[i] == 1) {
+                    andSelected[andSel++] = i;
+                  }
+                }
+              } else {
+                for (int i = 0; i != n; i++) {
+                  if (nextVector[i] == 1) {
+                    andSelected[andSel++] = i;
+                  }
+                }
+              }
+            }
+          } else {
+
+            /*
+             * Current child column can have nulls.
+             */
+
+            Preconditions.checkState(andSel == 0);
+            andRepeating = false;
+
+            if (andRepeatingIsNull) {
+
+              /*
+               * Since andRepeatingIsNull is true, we always set intermediateNulls when building
+               * andSel/andSelected when the next row is null or true...
+               */
+              if (batch.selectedInUse) {
+                for (int j = 0; j != n; j++) {
+                  int i = sel[j];
+                  if (nextColVector.isNull[i] || nextVector[i] == 1) {
+                    intermediateNulls[i] = true;
+                    andSelected[andSel++] = i;
+                  }
+                }
+              } else {
+                for (int i = 0; i != n; i++) {
+                  if (nextColVector.isNull[i] || nextVector[i] == 1) {
+                    intermediateNulls[i] = true;
+                    andSelected[andSel++] = i;
+                  }
+                }
+              }
+              andRepeatingIsNull = false;
+            } else {
+
+              /*
+               * Previous rounds were all true with no null child columns.  Build
+               * andSel/andSelected when the next row is true; also build when next is null
+               * and set intermediateNulls to true, too.
+               */
+              if (batch.selectedInUse) {
+                for (int j = 0; j != n; j++) {
+                  int i = sel[j];
+                  if (nextColVector.isNull[i]) {
+                    intermediateNulls[i] = true;
+                    andSelected[andSel++] = i;
+                  } else if (nextVector[i] == 1) {
+                    andSelected[andSel++] = i;
+                  }
+                }
+              } else {
+                for (int i = 0; i != n; i++) {
+                  if (nextColVector.isNull[i]) {
+                    intermediateNulls[i] = true;
+                    andSelected[andSel++] = i;
+                  } else if (nextVector[i] == 1) {
+                    andSelected[andSel++] = i;
+                  }
+                }
+              }
+            }
           }
         }
-        outV.isRepeating = false;
-      } else if (!inputColVector1.isRepeating && inputColVector2.isRepeating) {
-        if (batch.selectedInUse) {
-          for (int j = 0; j != n; j++) {
-            int i = sel[j];
-            outputVector[i] = vector1[i] & vector2Value;
-            outV.isNull[i] = inputColVector1.isNull[i] && (vector2[0] == 1);
-          }
-        } else {
-          for (int i = 0; i != n; i++) {
-            outputVector[i] = vector1[i] & vector2Value;
-            outV.isNull[i] = inputColVector1.isNull[i] && (vector2[0] == 1);
+      } else {
+
+        /*
+         * Continue in row mode: the andSel variable and andSelected member array contains the
+         * rows that are some combination of true and null.
+         */
+        if (nextColVector.isRepeating) {
+
+          /*
+           * Current child column is repeating which affects all rows.
+           */
+          if (nextColVector.noNulls || !nextColVector.isNull[0]) {
+
+            if (nextVector[0] == 0) {
+              // When the entire child column is repeating false, we are done for AND.
+              outV.isRepeating = true;
+              outputVector[0] = 0;
+              return;
+            } else {
+              // Child column is all true. Keep current andSel/andSelected rows.
+            }
+          } else {
+            Preconditions.checkState(nextColVector.isNull[0]);
+
+            // Column is repeating null -- need to mark all current rows in andSel/andSelected
+            // as null.
+            for (int j = 0; j < andSel; j++) {
+              int i = andSelected[j];
+              intermediateNulls[i] = true;
+            }
           }
-        }
-        outV.isRepeating = false;
-      } else /* neither side is repeating */{
-        if (batch.selectedInUse) {
-          for (int j = 0; j != n; j++) {
-            int i = sel[j];
-            outputVector[i] = vector1[i] & vector2[i];
-            outV.isNull[i] = inputColVector1.isNull[i] && (vector2[i] == 1);
+        } else if (nextColVector.noNulls) {
+
+          /*
+           * Current child column has no nulls.
+           */
+
+          /*
+           * Rebuild andSel/andSelected to keep true rows.
+           */
+          int newSel = 0;
+          for (int j = 0; j < andSel; j++) {
+            int i = andSelected[j];
+            if (nextVector[i] == 1) {
+              andSelected[newSel++] = i;
+            }
           }
+          andSel = newSel;
         } else {
-          for (int i = 0; i != n; i++) {
-            outputVector[i] = vector1[i] & vector2[i];
-            outV.isNull[i] = inputColVector1.isNull[i] && (vector2[i] == 1);
+
+          /*
+           * Current child column can have nulls.
+           */
+
+          /*
+           * Rebuild andSel/andSelected to keep true rows or null rows.
+           */
+          int newSel = 0;
+          for (int j = 0; j < andSel; j++) {
+            int i = andSelected[j];
+            if (nextColVector.isNull[i]) {
+              // At least one null.
+              intermediateNulls[i] = true;
+              andSelected[newSel++] = i;
+            } else if (nextVector[i] == 1) {
+              andSelected[newSel++] = i;
+            }
           }
+          andSel = newSel;
         }
-        outV.isRepeating = false;
       }
-      outV.noNulls = false;
-    } else /* !inputColVector1.noNulls && !inputColVector2.noNulls */{
-      // either input 1 or input 2 may have nulls
-      if ((inputColVector1.isRepeating) && (inputColVector2.isRepeating)) {
-        // All must be selected otherwise size would be zero
-        // Repeating property will not change.
-        outV.isRepeating = true;
-        outputVector[0] = vector1[0] & vector2[0];
-        outV.isNull[0] = ((vector1[0] == 1) && inputColVector2.isNull[0])
-            || (inputColVector1.isNull[0] && (vector2[0] == 1))
-            || (inputColVector1.isNull[0] && inputColVector2.isNull[0]);
-      } else if (inputColVector1.isRepeating && !inputColVector2.isRepeating) {
-        if (batch.selectedInUse) {
-          for (int j = 0; j != n; j++) {
-            int i = sel[j];
-            outputVector[i] = vector1Value & vector2[i];
-            outV.isNull[i] = ((vector1[0] == 1) && inputColVector2.isNull[i])
-                || (inputColVector1.isNull[0] && (vector2[i] == 1))
-                || (inputColVector1.isNull[0] && inputColVector2.isNull[i]);
-          }
-        } else {
-          for (int i = 0; i != n; i++) {
-            outputVector[i] = vector1Value & vector2[i];
-            outV.isNull[i] = ((vector1[0] == 1) && inputColVector2.isNull[i])
-                || (inputColVector1.isNull[0] && (vector2[i] == 1))
-                || (inputColVector1.isNull[0] && inputColVector2.isNull[i]);
-          }
-        }
-        outV.isRepeating = false;
-      } else if (!inputColVector1.isRepeating && inputColVector2.isRepeating) {
-        if (batch.selectedInUse) {
-          for (int j = 0; j != n; j++) {
-            int i = sel[j];
-            outputVector[i] = vector1[i] & vector2Value;
-            outV.isNull[i] = ((vector1[i] == 1) && inputColVector2.isNull[0])
-                || (inputColVector1.isNull[i] && (vector2[0] == 1))
-                || (inputColVector1.isNull[i] && inputColVector2.isNull[0]);
-          }
-        } else {
-          for (int i = 0; i != n; i++) {
-            outputVector[i] = vector1[i] & vector2Value;
-            outV.isNull[i] = ((vector1[i] == 1) && inputColVector2.isNull[0])
-                || (inputColVector1.isNull[i] && (vector2[0] == 1))
-                || (inputColVector1.isNull[i] && inputColVector2.isNull[0]);
+    } while (++colNum < colNums.length);
+
+    /*
+     * Produce final result.
+     */
+    if (andRepeating) {
+      outV.isRepeating = true;
+      if (andRepeatingIsNull) {
+        // The appearance of a null makes the repeated result null.
+        outV.noNulls = false;
+        outV.isNull[0] = true;
+      } else {
+        // All columns are repeating true.
+        outputVector[0] = 1;
+      }
+    } else if (andSel == 0) {
+      // No rows had true.
+      outV.isRepeating = true;
+      outputVector[0] = 0;
+    } else {
+      // Ok, rows were some combination of true and null throughout.
+      int andIndex = 0;
+      if (batch.selectedInUse) {
+        /*
+         * The batch selected array has all the rows we are projecting a boolean from.
+         * The andSelected array has a subset of the selected rows that have at least
+         * one true and may have some nulls. Now we need to decide if we are going to mark
+         * those rows as true, or null because there was at least one null.
+         *
+         * We use the andIndex to progress through the andSelected array and make a decision
+         * on how to fill out the boolean result.
+         *
+         * Since we reset the output column, we shouldn't have to set isNull false for true
+         * entries.
+         */
+        for (int j = 0; j != n; j++) {
+          int i = sel[j];
+          if (andIndex < andSel && andSelected[andIndex] == i) {
+            // We haven't processed all the andSelected entries and the row index is in
+            // andSelected, so make a result decision for true or null.
+            if (intermediateNulls[i]) {
+              outV.noNulls = false;
+              outV.isNull[i] = true;
+            } else {
+              outputVector[i] = 1;
+            }
+            andIndex++;
+          } else {
+            // The row is not in the andSelected array.  Result is false.
+            outputVector[i] = 0;
           }
         }
-        outV.isRepeating = false;
-      } else /* neither side is repeating */{
-        if (batch.selectedInUse) {
-          for (int j = 0; j != n; j++) {
-            int i = sel[j];
-            outputVector[i] = vector1[i] & vector2[i];
-            outV.isNull[i] = ((vector1[i] == 1) && inputColVector2.isNull[i])
-                || (inputColVector1.isNull[i] && (vector2[i] == 1))
-                || (inputColVector1.isNull[i] && inputColVector2.isNull[i]);
-          }
-        } else {
-          for (int i = 0; i != n; i++) {
-            outputVector[i] = vector1[i] & vector2[i];
-            outV.isNull[i] = ((vector1[i] == 1) && inputColVector2.isNull[i])
-                || (inputColVector1.isNull[i] && (vector2[i] == 1))
-                || (inputColVector1.isNull[i] && inputColVector2.isNull[i]);
+        Preconditions.checkState(andIndex == andSel);
+      } else {
+        /*
+         * The andSelected array has a subset of the selected rows that have at least
+         * one true and may have some nulls. Now we need to decide if we are going to mark
+         * those rows as true, or null because there was at least one null.
+         *
+         * Prefill the result as all false.  Then decide about the andSelected entries.
+         */
+        Arrays.fill(outputVector, 0, n, 0);
+        for (int j = 0; j < andSel; j++) {
+          int i = andSelected[j];
+          if (intermediateNulls[i]) {
+            outV.noNulls = false;
+            outV.isNull[i] = true;
+          } else {
+            outputVector[i] = 1;
           }
         }
-        outV.isRepeating = false;
       }
-      outV.noNulls = false;
     }
   }
 
@@ -293,22 +558,6 @@ public class ColAndCol extends VectorExpression {
     return "boolean";
   }
 
-  public int getColNum1() {
-    return colNum1;
-  }
-
-  public void setColNum1(int colNum1) {
-    this.colNum1 = colNum1;
-  }
-
-  public int getColNum2() {
-    return colNum2;
-  }
-
-  public void setColNum2(int colNum2) {
-    this.colNum2 = colNum2;
-  }
-
   public void setOutputColumn(int outputColumn) {
     this.outputColumn = outputColumn;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/9dd70f0d/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ColOrCol.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ColOrCol.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ColOrCol.java
index 60ed2d4..db33281 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ColOrCol.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ColOrCol.java
@@ -18,28 +18,38 @@
 
 package org.apache.hadoop.hive.ql.exec.vector.expressions;
 
+import java.util.Arrays;
+
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 
+import com.google.common.base.Preconditions;
+
 /**
- * This class performs OR expression on two input columns and stores,
- * the boolean output in a separate output column. The boolean values
- * are supposed to be represented as 0/1 in a long vector.
+ * Evaluate OR of 2 or more boolean columns and store the boolean result in the
+ * output boolean column.  This is a projection or result producing expression (as opposed to
+ * a filter expression).
+ *
+ * Some child boolean columns may be vector expressions evaluated into boolean scratch columns.
  */
 public class ColOrCol extends VectorExpression {
 
   private static final long serialVersionUID = 1L;
 
-  private int colNum1;
-  private int colNum2;
+  private int[] colNums;
   private int outputColumn;
+  private int[] mapToChildExpression;
+  private int[] orSelected;
+  private boolean[] intermediateNulls;
 
-  public ColOrCol(int colNum1, int colNum2, int outputColumn) {
+  public ColOrCol(int[] colNums, int outputColumn) {
     this();
-    this.colNum1 = colNum1;
-    this.colNum2 = colNum2;
+    this.colNums = colNums;
     this.outputColumn = outputColumn;
+    mapToChildExpression = null;
+    orSelected = new int[VectorizedRowBatch.DEFAULT_SIZE];
+    intermediateNulls = new boolean[VectorizedRowBatch.DEFAULT_SIZE];
   }
 
   public ColOrCol() {
@@ -49,240 +59,494 @@ public class ColOrCol extends VectorExpression {
   @Override
   public void evaluate(VectorizedRowBatch batch) {
 
+    Preconditions.checkState(colNums.length >= 2);
+
+    /*
+     * Vector child expressions will be omitted if they are existing boolean vector columns,
+     * so the child index does not necessarily index into the childExpressions.
+     * We construct a simple index map to the child expression in mapToChildExpression.
+     */
+    if (childExpressions != null && mapToChildExpression == null) {
+      // Some vector child expressions can be omitted (e.g. if they are existing boolean columns).
+      mapToChildExpression = new int [colNums.length];
+      int childIndex = 0;
+      for (int i = 0; i < childExpressions.length; i++) {
+        VectorExpression ve = childExpressions[i];
+        int outputColumn = ve.getOutputColumn();
+        while (outputColumn != colNums[childIndex]) {
+          mapToChildExpression[childIndex++] = -1;
+        }
+        mapToChildExpression[childIndex++] = i;
+      }
+      Preconditions.checkState(childIndex == colNums.length);
+    }
+
+    final int n = batch.size;
+    if (n <= 0) {
+      // Nothing to do
+      return;
+    }
+
     if (childExpressions != null) {
-      super.evaluateChildren(batch);
+      /*
+       * Evaluate first child expression.  Other child are conditionally evaluated later
+       * based on whether there is still a need for OR processing.
+       */
+      int childExpressionIndex = mapToChildExpression[0];
+      if (childExpressionIndex != -1) {
+        VectorExpression ve = childExpressions[childExpressionIndex];
+        Preconditions.checkState(ve.getOutputColumn() == colNums[0]);
+        ve.evaluate(batch);
+      }
     }
 
-    LongColumnVector inputColVector1 = (LongColumnVector) batch.cols[colNum1];
-    LongColumnVector inputColVector2 = (LongColumnVector) batch.cols[colNum2];
     int[] sel = batch.selected;
-    int n = batch.size;
-    long[] vector1 = inputColVector1.vector;
-    long[] vector2 = inputColVector2.vector;
 
     LongColumnVector outV = (LongColumnVector) batch.cols[outputColumn];
     long[] outputVector = outV.vector;
-    if (n <= 0) {
-      // Nothing to do
-      return;
+
+    /**
+     * Null processing complicates the algorithm here for Multi-OR.
+     *
+     * Any true --> true
+     * 0 or more false with 1 or more null --> result = null
+     * All false --> false
+     *
+     * For OR processing, we set the outputVector row to true rows as we find them.
+     * Once set for OR, the row result doesn't change.
+     *
+     * We remember nulls in the intermediateNulls so we can later we can mark the row as null
+     * instead of false if there was a null.
+     */
+
+    /*
+     * orRepeating will be true when all the children column vectors processed so far are
+     * some combination of repeating false and repeating null.
+     * orRepeatingIsNull will be true when there is at least one repeating null column.
+     */
+    boolean orRepeating = false;
+    boolean orRepeatingHasNulls = false;
+
+    /*
+     * The orSel variable and orSelected member array represent rows that have at have
+     * some combination of false and nulls.
+     */
+    int orSel = 0;
+
+    Arrays.fill(intermediateNulls, 0, VectorizedRowBatch.DEFAULT_SIZE, false);
+
+    // Reset noNulls to true, isNull to false, isRepeating to false.
+    outV.reset();
+
+    // Initially, set all rows to false.
+    if (batch.selectedInUse) {
+      for (int j = 0; j != n; j++) {
+        int i = sel[j];
+        outputVector[i] = 0;
+      }
+    } else {
+      Arrays.fill(outputVector, 0, n, 0);
     }
 
-    long vector1Value = vector1[0];
-    long vector2Value = vector2[0];
-    if (inputColVector1.noNulls && inputColVector2.noNulls) {
-      if ((inputColVector1.isRepeating) && (inputColVector2.isRepeating)) {
-        // All must be selected otherwise size would be zero
-        // Repeating property will not change.
-        outV.isRepeating = true;
-        outputVector[0] = vector1[0] | vector2[0];
-      } else if (inputColVector1.isRepeating && !inputColVector2.isRepeating) {
-        if (batch.selectedInUse) {
-          for (int j = 0; j != n; j++) {
-            int i = sel[j];
-            outputVector[i] = vector1Value | vector2[i];
-          }
+    LongColumnVector firstColVector = (LongColumnVector) batch.cols[colNums[0]];
+    long[] firstVector = firstColVector.vector;
+
+    /*
+     * We prime the pump by evaluating the first child to see if we are starting with
+     * orRepeating/orRepeatingHasNulls or we are starting with orSel/orSelected processing.
+     */
+    if (firstColVector.isRepeating) {
+      if (firstColVector.noNulls || !firstColVector.isNull[0]) {
+        if (firstVector[0] == 1) {
+          // When the entire child column is repeating true, we are done for OR.
+          outV.isRepeating = true;
+          outputVector[0] = 1;
+          return;
         } else {
-          for (int i = 0; i != n; i++) {
-            outputVector[i] = vector1Value | vector2[i];
-          }
+          // First column is repeating false.
         }
-        outV.isRepeating = false;
-      } else if (!inputColVector1.isRepeating && inputColVector2.isRepeating) {
-        if (batch.selectedInUse) {
-          for (int j = 0; j != n; j++) {
-            int i = sel[j];
-            outputVector[i] = vector1[i] | vector2Value;
-          }
-        } else {
-          for (int i = 0; i != n; i++) {
-            outputVector[i] = vector1[i] | vector2Value;
+      } else {
+        Preconditions.checkState(firstColVector.isNull[0]);
+
+        // At least one repeating null column.
+        orRepeatingHasNulls = true;
+      }
+      orRepeating = true;
+    } else if (firstColVector.noNulls) {
+
+      /*
+       * No nulls -- so all false rows go in orSel/orSelected.  Otherwise, when the row is true,
+       * mark the output row as true.
+       */
+      if (batch.selectedInUse) {
+        for (int j = 0; j != n; j++) {
+          int i = sel[j];
+          if (firstVector[i] == 0) {
+            orSelected[orSel++] = i;
+          } else {
+            outputVector[i] = 1;
           }
         }
-        outV.isRepeating = false;
-      } else /* neither side is repeating */{
-        if (batch.selectedInUse) {
-          for (int j = 0; j != n; j++) {
-            int i = sel[j];
-            outputVector[i] = vector1[i] | vector2[i];
-          }
-        } else {
-          for (int i = 0; i != n; i++) {
-            outputVector[i] = vector1[i] | vector2[i];
+      } else {
+        for (int i = 0; i != n; i++) {
+          if (firstVector[i] == 0) {
+            orSelected[orSel++] = i;
+          } else {
+            outputVector[i] = 1;
           }
         }
-        outV.isRepeating = false;
       }
-      outV.noNulls = true;
-    } else if (inputColVector1.noNulls && !inputColVector2.noNulls) {
-      // only input 2 side has nulls
-      if ((inputColVector1.isRepeating) && (inputColVector2.isRepeating)) {
-        // All must be selected otherwise size would be zero
-        // Repeating property will not change.
-        outV.isRepeating = true;
-        outputVector[0] = vector1[0] | vector2[0];
-        outV.isNull[0] = (vector1[0] == 0) && inputColVector2.isNull[0];
-      } else if (inputColVector1.isRepeating && !inputColVector2.isRepeating) {
-        if (batch.selectedInUse) {
-          for (int j = 0; j != n; j++) {
-            int i = sel[j];
-            outputVector[i] = vector1Value | vector2[i];
-            outV.isNull[i] = (vector1Value == 0) && inputColVector2.isNull[i];
-          }
-        } else {
-          for (int i = 0; i != n; i++) {
-            outputVector[i] = vector1Value | vector2[i];
-            outV.isNull[i] = (vector1Value == 0) && inputColVector2.isNull[i];
+    } else {
+
+      /*
+       * Can be nulls -- so all false rows and null rows go in orSel/orSelected.
+       * Remember nulls in our separate intermediateNulls array.  Otherwise, when the row is true,
+       * mark the output row as true.
+       */
+      if (batch.selectedInUse) {
+        for (int j = 0; j != n; j++) {
+          int i = sel[j];
+          if (firstColVector.isNull[i]) {
+            intermediateNulls[i] = true;
+            orSelected[orSel++] = i;
+          } else if (firstVector[i] == 0) {
+            orSelected[orSel++] = i;
+          } else {
+            outputVector[i] = 1;
           }
         }
-        outV.isRepeating = false;
-      } else if (!inputColVector1.isRepeating && inputColVector2.isRepeating) {
-        if (batch.selectedInUse) {
-          for (int j = 0; j != n; j++) {
-            int i = sel[j];
-            outputVector[i] = vector1[i] | vector2Value;
-            outV.isNull[i] = (vector1[i] == 0) && inputColVector2.isNull[0];
-          }
-        } else {
-          for (int i = 0; i != n; i++) {
-            outputVector[i] = vector1[i] | vector2Value;
-            outV.isNull[i] = (vector1[i] == 0) && inputColVector2.isNull[0];
+      } else {
+        for (int i = 0; i != n; i++) {
+          if (firstColVector.isNull[i]) {
+            intermediateNulls[i] = true;
+            orSelected[orSel++] = i;
+          } else if (firstVector[i] == 0) {
+            orSelected[orSel++] = i;
+          } else {
+            outputVector[i] = 1;
           }
         }
-        outV.isRepeating = false;
-      } else /* neither side is repeating */{
-        if (batch.selectedInUse) {
-          for (int j = 0; j != n; j++) {
-            int i = sel[j];
-            outputVector[i] = vector1[i] | vector2[i];
-            outV.isNull[i] = (vector1[i] == 0) && inputColVector2.isNull[i];
-          }
-        } else {
-          for (int i = 0; i != n; i++) {
-            outputVector[i] = vector1[i] | vector2[i];
-            outV.isNull[i] = (vector1[i] == 0) && inputColVector2.isNull[i];
+      }
+    }
+
+    /*
+     * Process child #2 and above.
+     */
+    int colNum = 1;
+    do {
+      if (!orRepeating && orSel == 0) {
+
+        /*
+         * Since orSelected and orSel represent currently false entries and there are none,
+         * then everything is true (how philosophical!).
+         */
+        break;
+      }
+
+      if (childExpressions != null) {
+        int childExpressionIndex = mapToChildExpression[colNum];
+        if (childExpressionIndex != -1) {
+          if (orRepeating) {
+            /*
+             * We need to start with a full evaluate on all [selected] rows.
+             */
+            VectorExpression ve = childExpressions[childExpressionIndex];
+            Preconditions.checkState(ve.getOutputColumn() == colNums[colNum]);
+            ve.evaluate(batch);
+          } else {
+            /*
+             * Evaluate next child expression.
+             * But only on the orSelected rows (i.e. current false or false with nulls rows).
+             */
+            boolean saveSelectedInUse = batch.selectedInUse;
+            int[] saveSelected = sel;
+            batch.selectedInUse = true;
+            batch.selected = orSelected;
+
+            VectorExpression ve = childExpressions[childExpressionIndex];
+            Preconditions.checkState(ve.getOutputColumn() == colNums[colNum]);
+            ve.evaluate(batch);
+
+            batch.selectedInUse = saveSelectedInUse;
+            batch.selected = saveSelected;
           }
         }
-        outV.isRepeating = false;
       }
-      outV.noNulls = false;
-    } else if (!inputColVector1.noNulls && inputColVector2.noNulls) {
-      // only input 1 side has nulls
-      if ((inputColVector1.isRepeating) && (inputColVector2.isRepeating)) {
-        // All must be selected otherwise size would be zero
-        // Repeating property will not change.
-        outV.isRepeating = true;
-        outputVector[0] = vector1[0] | vector2[0];
-        outV.isNull[0] = inputColVector1.isNull[0] && (vector2[0] == 0);
-      } else if (inputColVector1.isRepeating && !inputColVector2.isRepeating) {
-        if (batch.selectedInUse) {
-          for (int j = 0; j != n; j++) {
-            int i = sel[j];
-            outputVector[i] = vector1Value | vector2[i];
-            outV.isNull[i] = inputColVector1.isNull[0] && (vector2[i] == 0);
+
+      LongColumnVector nextColVector = (LongColumnVector) batch.cols[colNums[colNum]];
+      long[] nextVector = nextColVector.vector;
+
+      if (orRepeating) {
+
+        /*
+         * The orRepeating flag means the whole batch has some combination of repeating false
+         * columns and repeating null columns.
+         */
+        if (nextColVector.isRepeating) {
+
+          /*
+           * Current child column is repeating so stay in repeating mode.
+           */
+          if (nextColVector.noNulls || !nextColVector.isNull[0]) {
+            if (nextVector[0] == 1) {
+              outV.isRepeating = true;
+              outputVector[0] = 1;
+              return;
+            } else {
+              // The orRepeatingHasNulls flag goes on to the next stage, too.
+            }
+          } else {
+            Preconditions.checkState(nextColVector.isNull[0]);
+
+            // At least one repeating null column.
+            orRepeatingHasNulls = true;
           }
+          // Continue with orRepeating as true.
         } else {
-          for (int i = 0; i != n; i++) {
-            outputVector[i] = vector1Value | vector2[i];
-            outV.isNull[i] = inputColVector1.isNull[0] && (vector2[i] == 0);
+
+          /*
+           * Switch away from orRepeating/orRepeatingHasNulls and now represent individual rows in
+           * orSel/orSelected.
+           */
+          if (nextColVector.noNulls) {
+
+            /*
+             * Current child column has no nulls.
+             */
+
+            Preconditions.checkState(orSel == 0);
+            orRepeating = false;
+  
+            if (orRepeatingHasNulls) {
+
+              /*
+               * Since orRepeatingIsNull is true, we always set intermediateNulls when building
+               * orSel/orSelected when the next row is false.  Otherwise, when the row is true, mark
+               * the output row as true.
+               */
+              if (batch.selectedInUse) {
+                for (int j = 0; j != n; j++) {
+                  int i = sel[j];
+                  if (nextVector[i] == 0) {
+                    intermediateNulls[i] = true;
+                    orSelected[orSel++] = i;
+                  } else {
+                    outputVector[i] = 1;
+                  }
+                }
+              } else {
+                for (int i = 0; i != n; i++) {
+                  if (nextVector[i] == 0) {
+                    intermediateNulls[i] = true;
+                    orSelected[orSel++] = i;
+                  } else {
+                    outputVector[i] = 1;
+                  }
+                }
+              }
+              orRepeatingHasNulls = false;
+            } else {
+  
+              /*
+               * Previous rounds were all false with no null child columns.  Build
+               * orSel/orSelected when the next row is false.  Otherwise, when the row is true, mark
+               * the output row as true.
+               */
+              if (batch.selectedInUse) {
+                for (int j = 0; j != n; j++) {
+                  int i = sel[j];
+                  if (nextVector[i] == 0) {
+                    orSelected[orSel++] = i;
+                  } else {
+                    outputVector[i] = 1;
+                  }
+                }
+              } else {
+                for (int i = 0; i != n; i++) {
+                  if (nextVector[i] == 0) {
+                    orSelected[orSel++] = i;
+                  } else {
+                    outputVector[i] = 1;
+                  }
+                }
+              }
+            }
+          } else {
+
+            /*
+             * Current child column can have nulls.
+             */
+
+            Preconditions.checkState(orSel == 0);
+            orRepeating = false;
+  
+            if (orRepeatingHasNulls) {
+  
+              /*
+               * Since orRepeatingIsNull is true, we always set intermediateNulls when building
+               * orSel/orSelected when the next row is null or false.  Otherwise, when the row
+               * is true mark the output row as true.
+               */
+              if (batch.selectedInUse) {
+                for (int j = 0; j != n; j++) {
+                  int i = sel[j];
+                  if (nextColVector.isNull[i] || nextVector[i] == 0) {
+                    intermediateNulls[i] = true;
+                    orSelected[orSel++] = i;
+                  } else {
+                    outputVector[i] = 1;
+                  }
+                }
+              } else {
+                for (int i = 0; i != n; i++) {
+                  if (nextColVector.isNull[i] || nextVector[i] == 0) {
+                    intermediateNulls[i] = true;
+                    orSelected[orSel++] = i;
+                  } else {
+                    outputVector[i] = 1;
+                  }
+                }
+              }
+              orRepeatingHasNulls = false;
+            } else {
+  
+              /*
+               * Previous rounds were all true with no null child columns.  Build
+               * andSel/andSelected when the next row is true; also build when next is null
+               * and set intermediateNulls to true, too.  Otherwise, when the row
+               * is true mark the output row as true.
+               */
+              if (batch.selectedInUse) {
+                for (int j = 0; j != n; j++) {
+                  int i = sel[j];
+                  if (nextColVector.isNull[i]) {
+                    intermediateNulls[i] = true;
+                    orSelected[orSel++] = i;
+                  } else if (nextVector[i] == 0) {
+                    orSelected[orSel++] = i;
+                  } else {
+                    outputVector[i] = 1;
+                  }
+                }
+              } else {
+                for (int i = 0; i != n; i++) {
+                  if (nextColVector.isNull[i]) {
+                    intermediateNulls[i] = true;
+                    orSelected[orSel++] = i;
+                  } else if (nextVector[i] == 0) {
+                    orSelected[orSel++] = i;
+                  } else {
+                    outputVector[i] = 1;
+                  }
+                }
+              }
+            }
           }
         }
-        outV.isRepeating = false;
-      } else if (!inputColVector1.isRepeating && inputColVector2.isRepeating) {
-        if (batch.selectedInUse) {
-          for (int j = 0; j != n; j++) {
-            int i = sel[j];
-            outputVector[i] = vector1[i] | vector2Value;
-            outV.isNull[i] = inputColVector1.isNull[i] && (vector2Value == 0);
-          }
-        } else {
-          for (int i = 0; i != n; i++) {
-            outputVector[i] = vector1[i] | vector2Value;
-            outV.isNull[i] = inputColVector1.isNull[i] && (vector2Value == 0);
+      } else {
+
+        /*
+         * Continue in row mode: the orSel variable and orSelected member array contains the
+         * rows that are a combination of false and null.
+         */
+        if (nextColVector.isRepeating) {
+
+          if (nextColVector.noNulls || !nextColVector.isNull[0]) {
+
+            if (nextVector[0] == 1) {
+              // When the entire child column is repeating true, we are done for OR.
+              outV.isRepeating = true;
+              outputVector[0] = 1;
+              return;
+            } else {
+              // Child column is all false. Keep all orSel/orSelected rows.
+            }
+          } else {
+            Preconditions.checkState(nextColVector.isNull[0]);
+
+            // Column is repeating null -- need to mark all current rows in orSel/orSelected
+            // as null.
+            for (int j = 0; j < orSel; j++) {
+              int i = orSelected[j];
+              intermediateNulls[i] = true;
+            }
           }
-        }
-        outV.isRepeating = false;
-      } else /* neither side is repeating */{
-        if (batch.selectedInUse) {
-          for (int j = 0; j != n; j++) {
-            int i = sel[j];
-            outputVector[i] = vector1[i] | vector2[i];
-            outV.isNull[i] = inputColVector1.isNull[i] && (vector2[i] == 0);
+        } else if (nextColVector.noNulls) {
+
+          /*
+           * Current child column has no nulls.
+           */
+
+          /*
+           * Rebuild orSel/orSelected to keep false rows.  True rows get set in output vector.
+           */
+          int newSel = 0;
+          for (int j = 0; j < orSel; j++) {
+            int i = orSelected[j];
+            if (nextVector[i] == 0) {
+              orSelected[newSel++] = i;
+            } else {
+              outputVector[i] = 1;
+            }
           }
+          orSel = newSel;
         } else {
-          for (int i = 0; i != n; i++) {
-            outputVector[i] = vector1[i] | vector2[i];
-            outV.isNull[i] = inputColVector1.isNull[i] && (vector2[i] == 0);
+
+          /*
+           * Current child column can have nulls.
+           */
+
+          /*
+           * Rebuild orSel/orSelected to keep false rows or null rows.  True rows get set in
+           * output vector.
+           */
+          int newSel = 0;
+          for (int j = 0; j < orSel; j++) {
+            int i = orSelected[j];
+            if (nextColVector.isNull[i]) {
+              // Mark row has at least one null.
+              intermediateNulls[i] = true;
+              orSelected[newSel++] = i;
+            } else if (nextVector[i] == 0) {
+              orSelected[newSel++] = i;
+            } else {
+              outputVector[i] = 1;
+            }
           }
+          orSel = newSel;
         }
-        outV.isRepeating = false;
       }
-      outV.noNulls = false;
-    } else /* !inputColVector1.noNulls && !inputColVector2.noNulls */{
-      // either input 1 or input 2 may have nulls
-      if ((inputColVector1.isRepeating) && (inputColVector2.isRepeating)) {
-        // All must be selected otherwise size would be zero
-        // Repeating property will not change.
-        outV.isRepeating = true;
-        outputVector[0] = vector1[0] | vector2[0];
-        outV.isNull[0] = ((vector1[0] == 0) && inputColVector2.isNull[0])
-            || (inputColVector1.isNull[0] && (vector2[0] == 0))
-            || (inputColVector1.isNull[0] && inputColVector2.isNull[0]);
-      } else if (inputColVector1.isRepeating && !inputColVector2.isRepeating) {
-        if (batch.selectedInUse) {
-          for (int j = 0; j != n; j++) {
-            int i = sel[j];
-            outputVector[i] = vector1Value | vector2[i];
-            outV.isNull[i] = ((vector1[0] == 0) && inputColVector2.isNull[i])
-                || (inputColVector1.isNull[0] && (vector2[i] == 0))
-                || (inputColVector1.isNull[0] && inputColVector2.isNull[i]);
-          }
-        } else {
-          for (int i = 0; i != n; i++) {
-            outputVector[i] = vector1Value | vector2[i];
-            outV.isNull[i] = ((vector1[0] == 0) && inputColVector2.isNull[i])
-                || (inputColVector1.isNull[0] && (vector2[i] == 0))
-                || (inputColVector1.isNull[0] && inputColVector2.isNull[i]);
-          }
-        }
-        outV.isRepeating = false;
-      } else if (!inputColVector1.isRepeating && inputColVector2.isRepeating) {
-        if (batch.selectedInUse) {
-          for (int j = 0; j != n; j++) {
-            int i = sel[j];
-            outputVector[i] = vector1[i] | vector2Value;
-            outV.isNull[i] = ((vector1[i] == 0) && inputColVector2.isNull[0])
-                || (inputColVector1.isNull[i] && (vector2[0] == 0))
-                || (inputColVector1.isNull[i] && inputColVector2.isNull[0]);
-          }
-        } else {
-          for (int i = 0; i != n; i++) {
-            outputVector[i] = vector1[i] | vector2Value;
-            outV.isNull[i] = ((vector1[i] == 0) && inputColVector2.isNull[0])
-                || (inputColVector1.isNull[i] && (vector2[0] == 0))
-                || (inputColVector1.isNull[i] && inputColVector2.isNull[0]);
-          }
-        }
-        outV.isRepeating = false;
-      } else /* neither side is repeating */{
-        if (batch.selectedInUse) {
-          for (int j = 0; j != n; j++) {
-            int i = sel[j];
-            outputVector[i] = vector1[i] | vector2[i];
-            outV.isNull[i] = ((vector1[i] == 0) && inputColVector2.isNull[i])
-                || (inputColVector1.isNull[i] && (vector2[i] == 0))
-                || (inputColVector1.isNull[i] && inputColVector2.isNull[i]);
-          }
-        } else {
-          for (int i = 0; i != n; i++) {
-            outputVector[i] = vector1[i] | vector2[i];
-            outV.isNull[i] = ((vector1[i] == 0) && inputColVector2.isNull[i])
-                || (inputColVector1.isNull[i] && (vector2[i] == 0))
-                || (inputColVector1.isNull[i] && inputColVector2.isNull[i]);
-          }
+    } while (++colNum < colNums.length);
+
+    /*
+     *  Produce final result.
+     */
+    if (orRepeating) {
+
+      /*
+       * The orRepeating flags means the whole batch is false and may have nulls.
+       */
+      outV.isRepeating = true;
+      if (orRepeatingHasNulls) {
+        outV.noNulls = false;
+        outV.isNull[0] = true;
+      } else {
+        outputVector[0] = 0;
+      }
+    } else {
+
+      /*
+       * When were any intermediate nulls for a row, the result row will be null.
+       * Note the true entries were previously set in outputVector as we went along.
+       */
+      for (int j = 0; j < orSel; j++) {
+        int i = orSelected[j];
+        Preconditions.checkState(outputVector[i] == 0);
+        if (intermediateNulls[i]) {
+          outV.noNulls = false;
+          outV.isNull[i] = true;
         }
-        outV.isRepeating = false;
       }
-      outV.noNulls = false;
     }
   }
 
@@ -296,22 +560,6 @@ public class ColOrCol extends VectorExpression {
     return "boolean";
   }
 
-  public int getColNum1() {
-    return colNum1;
-  }
-
-  public void setColNum1(int colNum1) {
-    this.colNum1 = colNum1;
-  }
-
-  public int getColNum2() {
-    return colNum2;
-  }
-
-  public void setColNum2(int colNum2) {
-    this.colNum2 = colNum2;
-  }
-
   public void setOutputColumn(int outputColumn) {
     this.outputColumn = outputColumn;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/9dd70f0d/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizationContext.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizationContext.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizationContext.java
index bb37a04..a90dc30 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizationContext.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizationContext.java
@@ -622,8 +622,6 @@ public class TestVectorizationContext {
     assertEquals(veAnd.getClass(), ColAndCol.class);
     assertEquals(1, veAnd.getChildExpressions().length);
     assertEquals(veAnd.getChildExpressions()[0].getClass(), LongColGreaterLongScalar.class);
-    assertEquals(2, ((ColAndCol) veAnd).getColNum1());
-    assertEquals(1, ((ColAndCol) veAnd).getColNum2());
     assertEquals(3, ((ColAndCol) veAnd).getOutputColumn());
 
     //OR
@@ -647,8 +645,6 @@ public class TestVectorizationContext {
     assertEquals(veOr.getClass(), ColOrCol.class);
     assertEquals(1, veAnd.getChildExpressions().length);
     assertEquals(veAnd.getChildExpressions()[0].getClass(), LongColGreaterLongScalar.class);
-    assertEquals(2, ((ColOrCol) veOr).getColNum1());
-    assertEquals(1, ((ColOrCol) veOr).getColNum2());
     assertEquals(3, ((ColOrCol) veOr).getOutputColumn());
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/9dd70f0d/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorLogicalExpressions.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorLogicalExpressions.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorLogicalExpressions.java
index 7d54a9c..16c4198 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorLogicalExpressions.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorLogicalExpressions.java
@@ -40,7 +40,7 @@ public class TestVectorLogicalExpressions {
   @Test
   public void testLongColOrLongCol() {
     VectorizedRowBatch batch = getBatchThreeBooleanCols();
-    ColOrCol expr = new ColOrCol(0, 1, 2);
+    ColOrCol expr = new ColOrCol(new int[] {0, 1}, 2);
     LongColumnVector outCol = (LongColumnVector) batch.cols[2];
     expr.evaluate(batch);
     // verify
@@ -96,7 +96,7 @@ public class TestVectorLogicalExpressions {
   @Test
   public void testLongColAndLongCol() {
     VectorizedRowBatch batch = getBatchThreeBooleanCols();
-    ColAndCol expr = new ColAndCol(0, 1, 2);
+    ColAndCol expr = new ColAndCol(new int[] {0, 1}, 2);
     LongColumnVector outCol = (LongColumnVector) batch.cols[2];
     expr.evaluate(batch);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/9dd70f0d/ql/src/test/queries/clientpositive/vector_multi_and_projection.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/vector_multi_and_projection.q b/ql/src/test/queries/clientpositive/vector_multi_and_projection.q
new file mode 100644
index 0000000..21ad36e
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/vector_multi_and_projection.q
@@ -0,0 +1,196 @@
+set hive.cli.print.header=true;
+set hive.explain.user=false;
+SET hive.auto.convert.join=true;
+set hive.fetch.task.conversion=none;
+set hive.mapred.mode=nonstrict;
+
+-- SORT_QUERY_RESULTS
+
+create table vectortab2k(
+            t tinyint,
+            si smallint,
+            i int,
+            b bigint,
+            f float,
+            d double,
+            dc decimal(38,18),
+            bo boolean,
+            s string,
+            s2 string,
+            ts timestamp,
+            ts2 timestamp,
+            dt date)
+ROW FORMAT DELIMITED FIELDS TERMINATED BY '|'
+STORED AS TEXTFILE;
+
+LOAD DATA LOCAL INPATH '../../data/files/vectortab2k' OVERWRITE INTO TABLE vectortab2k;
+
+CREATE TABLE scratch AS SELECT t, si, i, b, bo FROM vectortab2k;
+INSERT INTO TABLE scratch VALUES (NULL, NULL, NULL, NULL, NULL);
+
+CREATE TABLE vectortab2k_orc STORED AS ORC AS SELECT * FROM scratch;
+
+SET hive.vectorized.execution.enabled=true;
+
+EXPLAIN
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t < 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc
+        order by t, si, i) as q;
+
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t < 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc
+        order by t, si, i) as q;
+
+EXPLAIN
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, b, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND i < 0 AND b > 0) as multi_and_col from vectortab2k_orc
+        order by t, si, i, b) as q;
+
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, b, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND i < 0 AND b > 0) as multi_and_col from vectortab2k_orc
+        order by t, si, i, b) as q;
+
+-- Use a boolean column rather than a column comparison expression.
+EXPLAIN
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo, b, (t < 0) as child1, (si > 0) as child2, (bo) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND bo AND b > 0) as multi_and_col from vectortab2k_orc
+        order by t, si, bo, b) as q;
+
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo, b, (t < 0) as child1, (si > 0) as child2, (bo) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND bo AND b > 0) as multi_and_col from vectortab2k_orc
+        order by t, si, bo, b) as q;
+
+-- With some filtering
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t < 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc
+        where pmod(t, 4) > 1
+        order by t, si, i) as q;
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, b, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND i < 0 AND b > 0) as multi_and_col from vectortab2k_orc
+        where pmod(t, 4) < 2
+        order by t, si, i, b) as q;
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo, b, (t < 0) as child1, (si > 0) as child2, (bo) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND bo AND b > 0) as multi_and_col from vectortab2k_orc
+        where pmod(b, 8) == 7
+        order by t, si, bo, b) as q;
+
+SET hive.vectorized.execution.enabled=false;
+
+CREATE TABLE scratch_repeat AS SELECT t, si, i, b, bo, 20 as t_repeat,
+     9000 as si_repeat, 9233320 as i_repeat, -823823999339992 as b_repeat, false as bo_repeat_false, true as bo_repeat_true FROM vectortab2k;
+
+-- The repeated columns ought to create repeated VectorizedRowBatch for those columns.
+-- And then when we do a comparison, we should generate a repeated boolean result.
+CREATE TABLE vectortab2k_orc_repeat STORED AS ORC AS SELECT * FROM scratch_repeat;
+
+SET hive.vectorized.execution.enabled=true;
+
+-- t_repeat > 0 should generate all true.
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, i, (t_repeat > 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_repeat > 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc_repeat
+        order by t_repeat, si, i) as q;
+
+-- t_repeat < 0 should generate all false.
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, i, (t_repeat < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_repeat < 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc_repeat
+        order by t_repeat, si, i) as q;
+
+-- Two repeated false columns at beginning...
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si_repeat, i, (t_repeat > 0) as child1, (si_repeat > 0) as child2, (i < 0) as child3, (t_repeat > 0 AND si_repeat > 0 AND i < 0) as multi_and_col from vectortab2k_orc_repeat
+        order by t_repeat, si_repeat, i) as q;
+
+-- si_repeat > 0 should generate all true.
+SELECT sum(hash(*)) FROM
+    (SELECT t, si_repeat, i, b_repeat, (t < 0 ) as child1, (si_repeat > 0) as child2, (i < 0) as child3, (b_repeat > 0) as child4, (t < 0 AND si_repeat > 0 AND i < 0 AND b_repeat > 0) as multi_and_col from vectortab2k_orc_repeat
+        order by t, si_repeat, i, b_repeat) as q;
+
+-- si_repeat < 0 should generate all false.
+SELECT sum(hash(*)) FROM
+    (SELECT t, si_repeat, i, b_repeat, (t < 0) as child1, (si_repeat < 0) as child2, (i < 0) as child3, (b_repeat > 0) as child4, (t < 0 AND si_repeat < 0 AND i < 0 AND b_repeat > 0) as multi_and_col from vectortab2k_orc_repeat
+        order by t, si_repeat, i, b_repeat) as q;
+
+-- Use a boolean column rather than a column comparison expression.
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo_repeat_false, b, (t < 0) as child1, (si > 0) as child2, (bo_repeat_false) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND bo_repeat_false AND b > 0) as multi_and_col from vectortab2k_orc_repeat
+        order by t, si, bo_repeat_false, b) as q;
+
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo_repeat_true, b, (t < 0) as child1, (si > 0) as child2, (bo_repeat_true) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND bo_repeat_true AND b > 0) as multi_and_col from vectortab2k_orc_repeat
+        order by t, si, bo_repeat_true, b) as q;
+
+-- With some filtering
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, i, (t_repeat > 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_repeat > 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc_repeat
+        where pmod(si, 4) = 0
+        order by t_repeat, si, i) as q;
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, i, (t_repeat < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_repeat < 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc_repeat
+        where pmod(si, 4) = 3
+        order by t_repeat, si, i) as q;
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si_repeat, i, (t_repeat > 0) as child1, (si_repeat > 0) as child2, (i < 0) as child3, (t_repeat > 0 AND si_repeat > 0 AND i < 0) as multi_and_col from vectortab2k_orc_repeat
+        where pmod(si, 4) != 3
+        order by t_repeat, si_repeat, i) as q;
+SELECT sum(hash(*)) FROM
+    (SELECT t, si_repeat, i, b_repeat, (t < 0 ) as child1, (si_repeat > 0) as child2, (i < 0) as child3, (b_repeat > 0) as child4, (t < 0 AND si_repeat > 0 AND i < 0 AND b_repeat > 0) as multi_and_col from vectortab2k_orc_repeat
+        where pmod(si, 4) < 2
+        order by t, si_repeat, i, b_repeat) as q;
+SELECT sum(hash(*)) FROM
+    (SELECT t, si_repeat, i, b_repeat, (t < 0) as child1, (si_repeat < 0) as child2, (i < 0) as child3, (b_repeat > 0) as child4, (t < 0 AND si_repeat < 0 AND i < 0 AND b_repeat > 0) as multi_and_col from vectortab2k_orc_repeat
+        where pmod(t, 4) = 0
+        order by t, si_repeat, i, b_repeat) as q;
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo_repeat_false, b, (t < 0) as child1, (si > 0) as child2, (bo_repeat_false) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND bo_repeat_false AND b > 0) as multi_and_col from vectortab2k_orc_repeat
+        where pmod(b, 4) > 1
+        order by t, si, bo_repeat_false, b) as q;
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo_repeat_true, b, (t < 0) as child1, (si > 0) as child2, (bo_repeat_true) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND bo_repeat_true AND b > 0) as multi_and_col from vectortab2k_orc_repeat
+        where pmod(si, 4) < 3
+        order by t, si, bo_repeat_true, b) as q;
+
+
+SET hive.vectorized.execution.enabled=false;
+
+CREATE TABLE scratch_null AS SELECT t, si, i, b, bo,
+     cast(null as tinyint) as t_null, cast(null as smallint) as si_null, cast(null as int) as i_null, cast(null as bigint) as b_null, cast(null as boolean) as bo_null FROM vectortab2k;
+
+-- The nulled columns ought to create repeated null VectorizedRowBatch for those columns.
+CREATE TABLE vectortab2k_orc_null STORED AS ORC AS SELECT * FROM scratch_null;
+
+SET hive.vectorized.execution.enabled=true;
+
+SELECT sum(hash(*)) FROM
+    (SELECT t_null, si, i, (t_null is null) as child1, (si > 0) as child2, (i < 0) as child3, (t_null is null AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc_null
+        order by t_null, si, i) as q;
+
+SELECT sum(hash(*)) FROM
+    (SELECT t_null, si, i, (t_null < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_null < 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc_null
+        order by t_null, si, i) as q;
+
+SELECT sum(hash(*)) FROM
+    (SELECT t_null, si_null, i, (t_null is null) as child1, (si_null > 0) as child2, (i < 0) as child3, (t_null is null AND si_null > 0 AND i < 0) as multi_and_col from vectortab2k_orc_null
+        order by t_null, si_null, i) as q;
+    
+SELECT sum(hash(*)) FROM
+    (SELECT t_null, si_null, i_null, (t_null is null) as child1, (si_null is null) as child2, (i_null is null) as child3, (t_null is null AND si_null is null AND i_null is null) as multi_and_col from vectortab2k_orc_null
+        order by t_null, si_null, i_null) as q;
+
+-- With some filtering
+SELECT sum(hash(*)) FROM
+    (SELECT t_null, si, i, (t_null is null) as child1, (si > 0) as child2, (i < 0) as child3, (t_null is null AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc_null
+        where pmod(i,4) = 3
+        order by t_null, si, i) as q;
+SELECT sum(hash(*)) FROM
+    (SELECT t_null, si, i, (t_null < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_null < 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc_null
+        where pmod(i,4) = 2
+        order by t_null, si, i) as q;
+SELECT sum(hash(*)) FROM
+    (SELECT t_null, si_null, i, (t_null is null) as child1, (si_null > 0) as child2, (i < 0) as child3, (t_null is null AND si_null > 0 AND i < 0) as multi_and_col from vectortab2k_orc_null
+        where pmod(i,4) != 3
+        order by t_null, si_null, i) as q;
+SELECT sum(hash(*)) FROM
+    (SELECT t_null, si_null, i_null, (t_null is null) as child1, (si_null is null) as child2, (i_null is null) as child3, (t_null is null AND si_null is null AND i_null is null) as multi_and_col from vectortab2k_orc_null
+        where pmod(i,4) < 3
+        order by t_null, si_null, i_null) as q;
+


[3/5] hive git commit: HIVE-13084: Vectorization add support for PROJECTION Multi-AND/OR (Matt McCline, reviewed by Sergey Shelukhin)

Posted by mm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/9dd70f0d/ql/src/test/results/clientpositive/tez/vector_multi_or_projection.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/vector_multi_or_projection.q.out b/ql/src/test/results/clientpositive/tez/vector_multi_or_projection.q.out
new file mode 100644
index 0000000..0174ca1
--- /dev/null
+++ b/ql/src/test/results/clientpositive/tez/vector_multi_or_projection.q.out
@@ -0,0 +1,821 @@
+PREHOOK: query: -- SORT_QUERY_RESULTS
+
+create table vectortab2k(
+            t tinyint,
+            si smallint,
+            i int,
+            b bigint,
+            f float,
+            d double,
+            dc decimal(38,18),
+            bo boolean,
+            s string,
+            s2 string,
+            ts timestamp,
+            ts2 timestamp,
+            dt date)
+ROW FORMAT DELIMITED FIELDS TERMINATED BY '|'
+STORED AS TEXTFILE
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@vectortab2k
+POSTHOOK: query: -- SORT_QUERY_RESULTS
+
+create table vectortab2k(
+            t tinyint,
+            si smallint,
+            i int,
+            b bigint,
+            f float,
+            d double,
+            dc decimal(38,18),
+            bo boolean,
+            s string,
+            s2 string,
+            ts timestamp,
+            ts2 timestamp,
+            dt date)
+ROW FORMAT DELIMITED FIELDS TERMINATED BY '|'
+STORED AS TEXTFILE
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@vectortab2k
+PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/vectortab2k' OVERWRITE INTO TABLE vectortab2k
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: default@vectortab2k
+POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/vectortab2k' OVERWRITE INTO TABLE vectortab2k
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: default@vectortab2k
+PREHOOK: query: CREATE TABLE scratch AS SELECT t, si, i, b, bo FROM vectortab2k
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@vectortab2k
+PREHOOK: Output: database:default
+PREHOOK: Output: default@scratch
+POSTHOOK: query: CREATE TABLE scratch AS SELECT t, si, i, b, bo FROM vectortab2k
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@vectortab2k
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@scratch
+POSTHOOK: Lineage: scratch.b SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: scratch.bo SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: scratch.i SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: scratch.si SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: scratch.t SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:t, type:tinyint, comment:null), ]
+t	si	i	b	bo
+PREHOOK: query: INSERT INTO TABLE scratch VALUES (NULL, NULL, NULL, NULL, NULL)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__1
+PREHOOK: Output: default@scratch
+POSTHOOK: query: INSERT INTO TABLE scratch VALUES (NULL, NULL, NULL, NULL, NULL)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__1
+POSTHOOK: Output: default@scratch
+POSTHOOK: Lineage: scratch.b EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col4, type:string, comment:), ]
+POSTHOOK: Lineage: scratch.bo EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col5, type:string, comment:), ]
+POSTHOOK: Lineage: scratch.i EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col3, type:string, comment:), ]
+POSTHOOK: Lineage: scratch.si EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+POSTHOOK: Lineage: scratch.t EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+_col0	_col1	_col2	_col3	_col4
+PREHOOK: query: CREATE TABLE vectortab2k_orc STORED AS ORC AS SELECT * FROM scratch
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@scratch
+PREHOOK: Output: database:default
+PREHOOK: Output: default@vectortab2k_orc
+POSTHOOK: query: CREATE TABLE vectortab2k_orc STORED AS ORC AS SELECT * FROM scratch
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@scratch
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@vectortab2k_orc
+POSTHOOK: Lineage: vectortab2k_orc.b SIMPLE [(scratch)scratch.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc.bo SIMPLE [(scratch)scratch.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc.i SIMPLE [(scratch)scratch.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc.si SIMPLE [(scratch)scratch.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc.t SIMPLE [(scratch)scratch.FieldSchema(name:t, type:tinyint, comment:null), ]
+scratch.t	scratch.si	scratch.i	scratch.b	scratch.bo
+PREHOOK: query: EXPLAIN
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t < 0 OR si > 0 OR i < 0) as multi_or_col from vectortab2k_orc
+        order by t, si, i) as q
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t < 0 OR si > 0 OR i < 0) as multi_or_col from vectortab2k_orc
+        order by t, si, i) as q
+POSTHOOK: type: QUERY
+Explain
+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:
+        Reducer 2 <- Map 1 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: vectortab2k_orc
+                  Statistics: Num rows: 2001 Data size: 45620 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: t (type: tinyint), si (type: smallint), i (type: int), (t < 0) (type: boolean), (si > 0) (type: boolean), (i < 0) (type: boolean), ((t < 0) or (si > 0) or (i < 0)) (type: boolean)
+                    outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
+                    Statistics: Num rows: 2001 Data size: 45620 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: int)
+                      sort order: +++
+                      Statistics: Num rows: 2001 Data size: 45620 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col3 (type: boolean), _col4 (type: boolean), _col5 (type: boolean), _col6 (type: boolean)
+            Execution mode: vectorized
+        Reducer 2 
+            Reduce Operator Tree:
+              Select Operator
+                expressions: hash(KEY.reducesinkkey0,KEY.reducesinkkey1,KEY.reducesinkkey2,VALUE._col0,VALUE._col1,VALUE._col2,VALUE._col3) (type: int)
+                outputColumnNames: _col0
+                Statistics: Num rows: 2001 Data size: 45620 Basic stats: COMPLETE Column stats: NONE
+                Group By Operator
+                  aggregations: sum(_col0)
+                  mode: complete
+                  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.SequenceFileInputFormat
+                        output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t < 0 OR si > 0 OR i < 0) as multi_or_col from vectortab2k_orc
+        order by t, si, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t < 0 OR si > 0 OR i < 0) as multi_or_col from vectortab2k_orc
+        order by t, si, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc
+#### A masked pattern was here ####
+c0
+-64411364734
+PREHOOK: query: EXPLAIN
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, b, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 OR si > 0 OR i < 0 OR b > 0) as multi_or_col from vectortab2k_orc
+        order by t, si, i, b) as q
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, b, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 OR si > 0 OR i < 0 OR b > 0) as multi_or_col from vectortab2k_orc
+        order by t, si, i, b) as q
+POSTHOOK: type: QUERY
+Explain
+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:
+        Reducer 2 <- Map 1 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: vectortab2k_orc
+                  Statistics: Num rows: 2001 Data size: 45620 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: t (type: tinyint), si (type: smallint), i (type: int), b (type: bigint), (t < 0) (type: boolean), (si > 0) (type: boolean), (i < 0) (type: boolean), (b > 0) (type: boolean), ((t < 0) or (si > 0) or (i < 0) or (b > 0)) (type: boolean)
+                    outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
+                    Statistics: Num rows: 2001 Data size: 45620 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint)
+                      sort order: ++++
+                      Statistics: Num rows: 2001 Data size: 45620 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col4 (type: boolean), _col5 (type: boolean), _col6 (type: boolean), _col7 (type: boolean), _col8 (type: boolean)
+            Execution mode: vectorized
+        Reducer 2 
+            Reduce Operator Tree:
+              Select Operator
+                expressions: hash(KEY.reducesinkkey0,KEY.reducesinkkey1,KEY.reducesinkkey2,KEY.reducesinkkey3,VALUE._col0,VALUE._col1,VALUE._col2,VALUE._col3,VALUE._col4) (type: int)
+                outputColumnNames: _col0
+                Statistics: Num rows: 2001 Data size: 45620 Basic stats: COMPLETE Column stats: NONE
+                Group By Operator
+                  aggregations: sum(_col0)
+                  mode: complete
+                  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.SequenceFileInputFormat
+                        output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, b, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 OR si > 0 OR i < 0 OR b > 0) as multi_or_col from vectortab2k_orc
+        order by t, si, i, b) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, b, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 OR si > 0 OR i < 0 OR b > 0) as multi_or_col from vectortab2k_orc
+        order by t, si, i, b) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc
+#### A masked pattern was here ####
+c0
+96890479010
+PREHOOK: query: -- Use a boolean column rather than a column comparison expression.
+EXPLAIN
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo, b, (t < 0) as child1, (si > 0) as child2, bo as child3, (b > 0) as child4, (t < 0 OR si > 0 OR bo OR b > 0) as multi_or_col from vectortab2k_orc
+        order by t, si, bo, b) as q
+PREHOOK: type: QUERY
+POSTHOOK: query: -- Use a boolean column rather than a column comparison expression.
+EXPLAIN
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo, b, (t < 0) as child1, (si > 0) as child2, bo as child3, (b > 0) as child4, (t < 0 OR si > 0 OR bo OR b > 0) as multi_or_col from vectortab2k_orc
+        order by t, si, bo, b) as q
+POSTHOOK: type: QUERY
+Explain
+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:
+        Reducer 2 <- Map 1 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: vectortab2k_orc
+                  Statistics: Num rows: 2001 Data size: 45620 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: t (type: tinyint), si (type: smallint), bo (type: boolean), b (type: bigint), (t < 0) (type: boolean), (si > 0) (type: boolean), (b > 0) (type: boolean), ((t < 0) or (si > 0) or bo or (b > 0)) (type: boolean)
+                    outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col7, _col8
+                    Statistics: Num rows: 2001 Data size: 45620 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: boolean), _col3 (type: bigint)
+                      sort order: ++++
+                      Statistics: Num rows: 2001 Data size: 45620 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col4 (type: boolean), _col5 (type: boolean), _col7 (type: boolean), _col8 (type: boolean)
+            Execution mode: vectorized
+        Reducer 2 
+            Reduce Operator Tree:
+              Select Operator
+                expressions: hash(KEY.reducesinkkey0,KEY.reducesinkkey1,KEY.reducesinkkey2,KEY.reducesinkkey3,VALUE._col0,VALUE._col1,KEY.reducesinkkey2,VALUE._col2,VALUE._col3) (type: int)
+                outputColumnNames: _col0
+                Statistics: Num rows: 2001 Data size: 45620 Basic stats: COMPLETE Column stats: NONE
+                Group By Operator
+                  aggregations: sum(_col0)
+                  mode: complete
+                  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.SequenceFileInputFormat
+                        output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo, b, (t < 0) as child1, (si > 0) as child2, bo as child3, (b > 0) as child4, (t < 0 OR si > 0 OR bo OR b > 0) as multi_or_col from vectortab2k_orc
+        order by t, si, bo, b) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo, b, (t < 0) as child1, (si > 0) as child2, bo as child3, (b > 0) as child4, (t < 0 OR si > 0 OR bo OR b > 0) as multi_or_col from vectortab2k_orc
+        order by t, si, bo, b) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc
+#### A masked pattern was here ####
+c0
+26141426137
+PREHOOK: query: -- With some filtering
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t < 0 OR si > 0 OR i < 0) as multi_or_col from vectortab2k_orc
+        where pmod(i,4) = 2
+        order by t, si, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc
+#### A masked pattern was here ####
+POSTHOOK: query: -- With some filtering
+SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t < 0 OR si > 0 OR i < 0) as multi_or_col from vectortab2k_orc
+        where pmod(i,4) = 2
+        order by t, si, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc
+#### A masked pattern was here ####
+c0
+-36533047758
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, b, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 OR si > 0 OR i < 0 OR b > 0) as multi_or_col from vectortab2k_orc
+        where pmod(si,4) < 2
+        order by t, si, i, b) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, i, b, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 OR si > 0 OR i < 0 OR b > 0) as multi_or_col from vectortab2k_orc
+        where pmod(si,4) < 2
+        order by t, si, i, b) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc
+#### A masked pattern was here ####
+c0
+41220563314
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo, b, (t < 0) as child1, (si > 0) as child2, bo as child3, (b > 0) as child4, (t < 0 OR si > 0 OR bo OR b > 0) as multi_or_col from vectortab2k_orc
+        where pmod(i,4) = 2
+        order by t, si, bo, b) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si, bo, b, (t < 0) as child1, (si > 0) as child2, bo as child3, (b > 0) as child4, (t < 0 OR si > 0 OR bo OR b > 0) as multi_or_col from vectortab2k_orc
+        where pmod(i,4) = 2
+        order by t, si, bo, b) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc
+#### A masked pattern was here ####
+c0
+13203204057
+PREHOOK: query: CREATE TABLE scratch_repeat AS SELECT t, si, i, b, bo,
+    20 as t_repeat, 9000 as si_repeat, 9233320 as i_repeat, -823823999339992 as b_repeat, false as bo_repeat_false, true as bo_repeat_true FROM vectortab2k
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@vectortab2k
+PREHOOK: Output: database:default
+PREHOOK: Output: default@scratch_repeat
+POSTHOOK: query: CREATE TABLE scratch_repeat AS SELECT t, si, i, b, bo,
+    20 as t_repeat, 9000 as si_repeat, 9233320 as i_repeat, -823823999339992 as b_repeat, false as bo_repeat_false, true as bo_repeat_true FROM vectortab2k
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@vectortab2k
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@scratch_repeat
+POSTHOOK: Lineage: scratch_repeat.b SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: scratch_repeat.b_repeat SIMPLE []
+POSTHOOK: Lineage: scratch_repeat.bo SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: scratch_repeat.bo_repeat_false SIMPLE []
+POSTHOOK: Lineage: scratch_repeat.bo_repeat_true SIMPLE []
+POSTHOOK: Lineage: scratch_repeat.i SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: scratch_repeat.i_repeat SIMPLE []
+POSTHOOK: Lineage: scratch_repeat.si SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: scratch_repeat.si_repeat SIMPLE []
+POSTHOOK: Lineage: scratch_repeat.t SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:t, type:tinyint, comment:null), ]
+POSTHOOK: Lineage: scratch_repeat.t_repeat SIMPLE []
+t	si	i	b	bo	t_repeat	si_repeat	i_repeat	b_repeat	bo_repeat_false	bo_repeat_true
+PREHOOK: query: -- The repeated columns ought to create repeated VectorizedRowBatch for those columns.
+-- And then when we do a comparison, we should generate a repeated boolean result.
+CREATE TABLE vectortab2k_orc_repeat STORED AS ORC AS SELECT * FROM scratch_repeat
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@scratch_repeat
+PREHOOK: Output: database:default
+PREHOOK: Output: default@vectortab2k_orc_repeat
+POSTHOOK: query: -- The repeated columns ought to create repeated VectorizedRowBatch for those columns.
+-- And then when we do a comparison, we should generate a repeated boolean result.
+CREATE TABLE vectortab2k_orc_repeat STORED AS ORC AS SELECT * FROM scratch_repeat
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@scratch_repeat
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@vectortab2k_orc_repeat
+POSTHOOK: Lineage: vectortab2k_orc_repeat.b SIMPLE [(scratch_repeat)scratch_repeat.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_repeat.b_repeat SIMPLE [(scratch_repeat)scratch_repeat.FieldSchema(name:b_repeat, type:bigint, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_repeat.bo SIMPLE [(scratch_repeat)scratch_repeat.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_repeat.bo_repeat_false SIMPLE [(scratch_repeat)scratch_repeat.FieldSchema(name:bo_repeat_false, type:boolean, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_repeat.bo_repeat_true SIMPLE [(scratch_repeat)scratch_repeat.FieldSchema(name:bo_repeat_true, type:boolean, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_repeat.i SIMPLE [(scratch_repeat)scratch_repeat.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_repeat.i_repeat SIMPLE [(scratch_repeat)scratch_repeat.FieldSchema(name:i_repeat, type:int, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_repeat.si SIMPLE [(scratch_repeat)scratch_repeat.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_repeat.si_repeat SIMPLE [(scratch_repeat)scratch_repeat.FieldSchema(name:si_repeat, type:int, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_repeat.t SIMPLE [(scratch_repeat)scratch_repeat.FieldSchema(name:t, type:tinyint, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_repeat.t_repeat SIMPLE [(scratch_repeat)scratch_repeat.FieldSchema(name:t_repeat, type:int, comment:null), ]
+scratch_repeat.t	scratch_repeat.si	scratch_repeat.i	scratch_repeat.b	scratch_repeat.bo	scratch_repeat.t_repeat	scratch_repeat.si_repeat	scratch_repeat.i_repeat	scratch_repeat.b_repeat	scratch_repeat.bo_repeat_false	scratch_repeat.bo_repeat_true
+PREHOOK: query: -- t_repeat < 0 should generate all false.
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, i, (t_repeat < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_repeat < 0 OR si > 0 OR i < 0) as multi_or_col from vectortab2k_orc_repeat
+        order by t_repeat, si, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: -- t_repeat < 0 should generate all false.
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, i, (t_repeat < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_repeat < 0 OR si > 0 OR i < 0) as multi_or_col from vectortab2k_orc_repeat
+        order by t_repeat, si, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+-53707877629
+PREHOOK: query: -- t_repeat > 0 should generate all true.
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, i, (t_repeat > 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_repeat > 0 OR si > 0 OR i < 0) as multi_or_col from vectortab2k_orc_repeat
+        order by t_repeat, si, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: -- t_repeat > 0 should generate all true.
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, i, (t_repeat > 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_repeat > 0 OR si > 0 OR i < 0) as multi_or_col from vectortab2k_orc_repeat
+        order by t_repeat, si, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+-53648295050
+PREHOOK: query: -- Two repeated false columns at beginning...
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si_repeat, i, (t_repeat < 0) as child1, (si_repeat < 0) as child2, (i < 0) as child3, (t_repeat < 0 OR si_repeat < 0 OR i < 0) as multi_or_col from vectortab2k_orc_repeat
+        order by t_repeat, si_repeat, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: -- Two repeated false columns at beginning...
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si_repeat, i, (t_repeat < 0) as child1, (si_repeat < 0) as child2, (i < 0) as child3, (t_repeat < 0 OR si_repeat < 0 OR i < 0) as multi_or_col from vectortab2k_orc_repeat
+        order by t_repeat, si_repeat, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+-55182904269
+PREHOOK: query: -- si_repeat > 0 should generate all true.
+SELECT sum(hash(*)) FROM
+    (SELECT t, si_repeat, i, b, (t < 0) as child1, (si_repeat > 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 OR si_repeat > 0 OR i < 0 OR b > 0) as multi_or_col from vectortab2k_orc_repeat
+        order by t, si_repeat, i, b) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: -- si_repeat > 0 should generate all true.
+SELECT sum(hash(*)) FROM
+    (SELECT t, si_repeat, i, b, (t < 0) as child1, (si_repeat > 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 OR si_repeat > 0 OR i < 0 OR b > 0) as multi_or_col from vectortab2k_orc_repeat
+        order by t, si_repeat, i, b) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+33185964172
+PREHOOK: query: -- si_repeat < 0 should generate all false.
+SELECT sum(hash(*)) FROM
+    (SELECT t, si_repeat, i, b, (t < 0) as child1, (si_repeat < 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 OR si_repeat < 0 OR i < 0 OR b > 0) as multi_or_col from vectortab2k_orc_repeat
+        order by t, si_repeat, i, b) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: -- si_repeat < 0 should generate all false.
+SELECT sum(hash(*)) FROM
+    (SELECT t, si_repeat, i, b, (t < 0) as child1, (si_repeat < 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 OR si_repeat < 0 OR i < 0 OR b > 0) as multi_or_col from vectortab2k_orc_repeat
+        order by t, si_repeat, i, b) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+33126382002
+PREHOOK: query: -- Use a boolean column rather than a column comparison expression.
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, bo_repeat_false, b, (t_repeat > 0) as child1, (si > 0) as child2, (bo_repeat_false) as child3, (b > 0) as child4, (t_repeat > 0 OR si > 0 OR bo_repeat_false OR b > 0) as multi_or_col from vectortab2k_orc_repeat
+        order by t_repeat, si, bo_repeat_false, b) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: -- Use a boolean column rather than a column comparison expression.
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, bo_repeat_false, b, (t_repeat > 0) as child1, (si > 0) as child2, (bo_repeat_false) as child3, (b > 0) as child4, (t_repeat > 0 OR si > 0 OR bo_repeat_false OR b > 0) as multi_or_col from vectortab2k_orc_repeat
+        order by t_repeat, si, bo_repeat_false, b) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+-35841267581
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, bo_repeat_true, b, (t_repeat > 0) as child1, (si > 0) as child2, (bo_repeat_true) as child3, (b > 0) as child4, (t_repeat > 0 OR si > 0 OR bo_repeat_true OR b > 0) as multi_or_col from vectortab2k_orc_repeat
+        order by t_repeat, si, bo_repeat_true, b) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, bo_repeat_true, b, (t_repeat > 0) as child1, (si > 0) as child2, (bo_repeat_true) as child3, (b > 0) as child4, (t_repeat > 0 OR si > 0 OR bo_repeat_true OR b > 0) as multi_or_col from vectortab2k_orc_repeat
+        order by t_repeat, si, bo_repeat_true, b) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+85605607459
+PREHOOK: query: -- With some filtering
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, i, (t_repeat < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_repeat < 0 OR si > 0 OR i < 0) as multi_or_col from vectortab2k_orc_repeat
+        where pmod(i,4) > 1
+        order by t_repeat, si, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: -- With some filtering
+SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, i, (t_repeat < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_repeat < 0 OR si > 0 OR i < 0) as multi_or_col from vectortab2k_orc_repeat
+        where pmod(i,4) > 1
+        order by t_repeat, si, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+-30735538712
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, i, (t_repeat > 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_repeat > 0 OR si > 0 OR i < 0) as multi_or_col from vectortab2k_orc_repeat
+        where pmod(si,4) > 1
+        order by t_repeat, si, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, i, (t_repeat > 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_repeat > 0 OR si > 0 OR i < 0) as multi_or_col from vectortab2k_orc_repeat
+        where pmod(si,4) > 1
+        order by t_repeat, si, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+-16221803790
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si_repeat, i, (t_repeat < 0) as child1, (si_repeat < 0) as child2, (i < 0) as child3, (t_repeat < 0 OR si_repeat < 0 OR i < 0) as multi_or_col from vectortab2k_orc_repeat
+        where pmod(i,4) > 1
+        order by t_repeat, si_repeat, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si_repeat, i, (t_repeat < 0) as child1, (si_repeat < 0) as child2, (i < 0) as child3, (t_repeat < 0 OR si_repeat < 0 OR i < 0) as multi_or_col from vectortab2k_orc_repeat
+        where pmod(i,4) > 1
+        order by t_repeat, si_repeat, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+-56285878286
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si_repeat, i, b, (t < 0) as child1, (si_repeat > 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 OR si_repeat > 0 OR i < 0 OR b > 0) as multi_or_col from vectortab2k_orc_repeat
+        where pmod(t,4) > 1
+        order by t, si_repeat, i, b) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si_repeat, i, b, (t < 0) as child1, (si_repeat > 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 OR si_repeat > 0 OR i < 0 OR b > 0) as multi_or_col from vectortab2k_orc_repeat
+        where pmod(t,4) > 1
+        order by t, si_repeat, i, b) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+-7888615123
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si_repeat, i, b, (t < 0) as child1, (si_repeat < 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 OR si_repeat < 0 OR i < 0 OR b > 0) as multi_or_col from vectortab2k_orc_repeat
+        where pmod(b,4) > 1
+        order by t, si_repeat, i, b) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si_repeat, i, b, (t < 0) as child1, (si_repeat < 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 OR si_repeat < 0 OR i < 0 OR b > 0) as multi_or_col from vectortab2k_orc_repeat
+        where pmod(b,4) > 1
+        order by t, si_repeat, i, b) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+16350664785
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, bo_repeat_false, b, (t_repeat > 0) as child1, (si > 0) as child2, (bo_repeat_false) as child3, (b > 0) as child4, (t_repeat > 0 OR si > 0 OR bo_repeat_false OR b > 0) as multi_or_col from vectortab2k_orc_repeat
+        where pmod(si,4) > 1
+        order by t_repeat, si, bo_repeat_false, b) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, bo_repeat_false, b, (t_repeat > 0) as child1, (si > 0) as child2, (bo_repeat_false) as child3, (b > 0) as child4, (t_repeat > 0 OR si > 0 OR bo_repeat_false OR b > 0) as multi_or_col from vectortab2k_orc_repeat
+        where pmod(si,4) > 1
+        order by t_repeat, si, bo_repeat_false, b) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+-5022644323
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, bo_repeat_true, b, (t_repeat > 0) as child1, (si > 0) as child2, (bo_repeat_true) as child3, (b > 0) as child4, (t_repeat > 0 OR si > 0 OR bo_repeat_true OR b > 0) as multi_or_col from vectortab2k_orc_repeat
+        where pmod(si,4) < 2
+        order by t_repeat, si, bo_repeat_true, b) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_repeat, si, bo_repeat_true, b, (t_repeat > 0) as child1, (si > 0) as child2, (bo_repeat_true) as child3, (b > 0) as child4, (t_repeat > 0 OR si > 0 OR bo_repeat_true OR b > 0) as multi_or_col from vectortab2k_orc_repeat
+        where pmod(si,4) < 2
+        order by t_repeat, si, bo_repeat_true, b) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_repeat
+#### A masked pattern was here ####
+c0
+51133657254
+PREHOOK: query: CREATE TABLE scratch_null AS SELECT t, si, i, b, bo,
+     cast(null as tinyint) as t_null, cast(null as smallint) as si_null, cast(null as int) as i_null, cast(null as bigint) as b_null, cast(null as boolean) as bo_null FROM vectortab2k
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@vectortab2k
+PREHOOK: Output: database:default
+PREHOOK: Output: default@scratch_null
+POSTHOOK: query: CREATE TABLE scratch_null AS SELECT t, si, i, b, bo,
+     cast(null as tinyint) as t_null, cast(null as smallint) as si_null, cast(null as int) as i_null, cast(null as bigint) as b_null, cast(null as boolean) as bo_null FROM vectortab2k
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@vectortab2k
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@scratch_null
+POSTHOOK: Lineage: scratch_null.b SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: scratch_null.b_null SIMPLE []
+POSTHOOK: Lineage: scratch_null.bo SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: scratch_null.bo_null SIMPLE []
+POSTHOOK: Lineage: scratch_null.i SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: scratch_null.i_null SIMPLE []
+POSTHOOK: Lineage: scratch_null.si SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: scratch_null.si_null SIMPLE []
+POSTHOOK: Lineage: scratch_null.t SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:t, type:tinyint, comment:null), ]
+POSTHOOK: Lineage: scratch_null.t_null SIMPLE []
+t	si	i	b	bo	t_null	si_null	i_null	b_null	bo_null
+PREHOOK: query: -- The nulled columns ought to create repeated null VectorizedRowBatch for those columns.
+CREATE TABLE vectortab2k_orc_null STORED AS ORC AS SELECT * FROM scratch_null
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@scratch_null
+PREHOOK: Output: database:default
+PREHOOK: Output: default@vectortab2k_orc_null
+POSTHOOK: query: -- The nulled columns ought to create repeated null VectorizedRowBatch for those columns.
+CREATE TABLE vectortab2k_orc_null STORED AS ORC AS SELECT * FROM scratch_null
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@scratch_null
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@vectortab2k_orc_null
+POSTHOOK: Lineage: vectortab2k_orc_null.b SIMPLE [(scratch_null)scratch_null.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_null.b_null SIMPLE [(scratch_null)scratch_null.FieldSchema(name:b_null, type:bigint, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_null.bo SIMPLE [(scratch_null)scratch_null.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_null.bo_null SIMPLE [(scratch_null)scratch_null.FieldSchema(name:bo_null, type:boolean, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_null.i SIMPLE [(scratch_null)scratch_null.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_null.i_null SIMPLE [(scratch_null)scratch_null.FieldSchema(name:i_null, type:int, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_null.si SIMPLE [(scratch_null)scratch_null.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_null.si_null SIMPLE [(scratch_null)scratch_null.FieldSchema(name:si_null, type:smallint, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_null.t SIMPLE [(scratch_null)scratch_null.FieldSchema(name:t, type:tinyint, comment:null), ]
+POSTHOOK: Lineage: vectortab2k_orc_null.t_null SIMPLE [(scratch_null)scratch_null.FieldSchema(name:t_null, type:tinyint, comment:null), ]
+scratch_null.t	scratch_null.si	scratch_null.i	scratch_null.b	scratch_null.bo	scratch_null.t_null	scratch_null.si_null	scratch_null.i_null	scratch_null.b_null	scratch_null.bo_null
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_null, si, i, (t_null is null) as child1, (si is null) as child2, (i < 0) as child3, (t_null is null OR si is null OR i < 0) as multi_and_col from vectortab2k_orc_null
+        order by t_null, si, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_null, si, i, (t_null is null) as child1, (si is null) as child2, (i < 0) as child3, (t_null is null OR si is null OR i < 0) as multi_and_col from vectortab2k_orc_null
+        order by t_null, si, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+c0
+-77366438890
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_null, si, i, (t_null < 0) as child1, (si > 0) as child2, (i is null) as child3, (t_null < 0 OR si > 0 OR i is null) as multi_and_col from vectortab2k_orc_null
+        order by t_null, si, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_null, si, i, (t_null < 0) as child1, (si > 0) as child2, (i is null) as child3, (t_null < 0 OR si > 0 OR i is null) as multi_and_col from vectortab2k_orc_null
+        order by t_null, si, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+c0
+-77425279225
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_null, si, i_null, (t_null < 0) as child1, (si > 0) as child2, (i_null < 0) as child3, (t_null < 0 OR si > 0 OR i_null < 0) as multi_and_col from vectortab2k_orc_null
+        order by t_null, si, i_null) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_null, si, i_null, (t_null < 0) as child1, (si > 0) as child2, (i_null < 0) as child3, (t_null < 0 OR si > 0 OR i_null < 0) as multi_and_col from vectortab2k_orc_null
+        order by t_null, si, i_null) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+c0
+33196722176
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si_null, i_null, (t < 0) as child1, (si_null > 0) as child2, (i_null < 0) as child3, (t < 0 OR si_null > 0 OR i_null < 0) as multi_and_col from vectortab2k_orc_null
+        order by t, si_null, i_null) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si_null, i_null, (t < 0) as child1, (si_null > 0) as child2, (i_null < 0) as child3, (t < 0 OR si_null > 0 OR i_null < 0) as multi_and_col from vectortab2k_orc_null
+        order by t, si_null, i_null) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+c0
+25898790496
+PREHOOK: query: -- With some filtering
+SELECT sum(hash(*)) FROM
+    (SELECT t_null, si, i, (t_null is null) as child1, (si is null) as child2, (i < 0) as child3, (t_null is null OR si is null OR i < 0) as multi_and_col from vectortab2k_orc_null
+        where pmod(i,4) = 2
+        order by t_null, si, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+POSTHOOK: query: -- With some filtering
+SELECT sum(hash(*)) FROM
+    (SELECT t_null, si, i, (t_null is null) as child1, (si is null) as child2, (i < 0) as child3, (t_null is null OR si is null OR i < 0) as multi_and_col from vectortab2k_orc_null
+        where pmod(i,4) = 2
+        order by t_null, si, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+c0
+-13477536069
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_null, si, i, (t_null < 0) as child1, (si > 0) as child2, (i is null) as child3, (t_null < 0 OR si > 0 OR i is null) as multi_and_col from vectortab2k_orc_null
+        where pmod(i,4) = 3
+        order by t_null, si, i) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_null, si, i, (t_null < 0) as child1, (si > 0) as child2, (i is null) as child3, (t_null < 0 OR si > 0 OR i is null) as multi_and_col from vectortab2k_orc_null
+        where pmod(i,4) = 3
+        order by t_null, si, i) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+c0
+-48784614557
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_null, si_null, i_null, (t_null < 0) as child1, (si > 0) as child2, (i_null < 0) as child3, (t_null < 0 OR si > 0 OR i_null < 0) as multi_and_col from vectortab2k_orc_null
+        where pmod(i,4) = 0
+        order by t_null, si_null, i_null) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t_null, si_null, i_null, (t_null < 0) as child1, (si > 0) as child2, (i_null < 0) as child3, (t_null < 0 OR si > 0 OR i_null < 0) as multi_and_col from vectortab2k_orc_null
+        where pmod(i,4) = 0
+        order by t_null, si_null, i_null) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+c0
+223184
+PREHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si_null, i_null, (t < 0) as child1, (si_null > 0) as child2, (i_null < 0) as child3, (t < 0 OR si_null > 0 OR i_null < 0) as multi_and_col from vectortab2k_orc_null
+        where pmod(i,4) != 2
+        order by t, si_null, i_null) as q
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(*)) FROM
+    (SELECT t, si_null, i_null, (t < 0) as child1, (si_null > 0) as child2, (i_null < 0) as child3, (t < 0 OR si_null > 0 OR i_null < 0) as multi_and_col from vectortab2k_orc_null
+        where pmod(i,4) != 2
+        order by t, si_null, i_null) as q
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k_orc_null
+#### A masked pattern was here ####
+c0
+6738993743