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 2015/11/01 05:32:51 UTC

[06/10] hive git commit: HIVE-12290 Native Vector ReduceSink (Matt McCline, reviewed by Gopal V)

http://git-wip-us.apache.org/repos/asf/hive/blob/409db57d/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorReduceSinkInfo.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorReduceSinkInfo.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorReduceSinkInfo.java
new file mode 100644
index 0000000..8c35415
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorReduceSinkInfo.java
@@ -0,0 +1,123 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.plan;
+
+import org.apache.hadoop.hive.ql.exec.vector.ColumnVector.Type;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+
+/**
+ * VectorGroupByAggregrationInfo.
+ *
+ * A convenience data structure that has information needed to vectorize reduce sink.
+ *
+ * It is created by the Vectorizer when it is determining whether it can specialize so the
+ * information doesn't have to be recreated again and agains by the VectorReduceSinkOperator's
+ * constructors and later during execution.
+ */
+public class VectorReduceSinkInfo {
+
+  private static long serialVersionUID = 1L;
+
+  private int[] reduceSinkKeyColumnMap;
+  private TypeInfo[] reduceSinkKeyTypeInfos;
+  private Type[] reduceSinkKeyColumnVectorTypes;
+  private VectorExpression[] reduceSinkKeyExpressions;
+
+  private int[] reduceSinkValueColumnMap;
+  private TypeInfo[] reduceSinkValueTypeInfos;
+  private Type[] reduceSinkValueColumnVectorTypes;
+  private VectorExpression[] reduceSinkValueExpressions;
+
+  public VectorReduceSinkInfo() {
+    reduceSinkKeyColumnMap = null;
+    reduceSinkKeyTypeInfos = null;
+    reduceSinkKeyColumnVectorTypes = null;
+    reduceSinkKeyExpressions = null;
+
+    reduceSinkValueColumnMap = null;
+    reduceSinkValueTypeInfos = null;
+    reduceSinkValueColumnVectorTypes = null;
+    reduceSinkValueExpressions = null;
+  }
+
+  public int[] getReduceSinkKeyColumnMap() {
+    return reduceSinkKeyColumnMap;
+  }
+
+  public void setReduceSinkKeyColumnMap(int[] reduceSinkKeyColumnMap) {
+    this.reduceSinkKeyColumnMap = reduceSinkKeyColumnMap;
+  }
+
+  public TypeInfo[] getReduceSinkKeyTypeInfos() {
+    return reduceSinkKeyTypeInfos;
+  }
+
+  public void setReduceSinkKeyTypeInfos(TypeInfo[] reduceSinkKeyTypeInfos) {
+    this.reduceSinkKeyTypeInfos = reduceSinkKeyTypeInfos;
+  }
+
+  public Type[] getReduceSinkKeyColumnVectorTypes() {
+    return reduceSinkKeyColumnVectorTypes;
+  }
+
+  public void setReduceSinkKeyColumnVectorTypes(Type[] reduceSinkKeyColumnVectorTypes) {
+    this.reduceSinkKeyColumnVectorTypes = reduceSinkKeyColumnVectorTypes;
+  }
+
+  public VectorExpression[] getReduceSinkKeyExpressions() {
+    return reduceSinkKeyExpressions;
+  }
+
+  public void setReduceSinkKeyExpressions(VectorExpression[] reduceSinkKeyExpressions) {
+    this.reduceSinkKeyExpressions = reduceSinkKeyExpressions;
+  }
+
+  public int[] getReduceSinkValueColumnMap() {
+    return reduceSinkValueColumnMap;
+  }
+
+  public void setReduceSinkValueColumnMap(int[] reduceSinkValueColumnMap) {
+    this.reduceSinkValueColumnMap = reduceSinkValueColumnMap;
+  }
+
+  public TypeInfo[] getReduceSinkValueTypeInfos() {
+    return reduceSinkValueTypeInfos;
+  }
+
+  public void setReduceSinkValueTypeInfos(TypeInfo[] reduceSinkValueTypeInfos) {
+    this.reduceSinkValueTypeInfos = reduceSinkValueTypeInfos;
+  }
+
+  public Type[] getReduceSinkValueColumnVectorTypes() {
+    return reduceSinkValueColumnVectorTypes;
+  }
+
+  public void setReduceSinkValueColumnVectorTypes(Type[] reduceSinkValueColumnVectorTypes) {
+    this.reduceSinkValueColumnVectorTypes = reduceSinkValueColumnVectorTypes;
+  }
+
+  public VectorExpression[] getReduceSinkValueExpressions() {
+    return reduceSinkValueExpressions;
+  }
+
+  public void setReduceSinkValueExpressions(VectorExpression[] reduceSinkValueExpressions) {
+    this.reduceSinkValueExpressions = reduceSinkValueExpressions;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/409db57d/ql/src/test/queries/clientpositive/vector_reduce1.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/vector_reduce1.q b/ql/src/test/queries/clientpositive/vector_reduce1.q
new file mode 100644
index 0000000..6284d76
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/vector_reduce1.q
@@ -0,0 +1,47 @@
+set hive.explain.user=false;
+SET hive.vectorized.execution.enabled=true;
+SET hive.vectorized.execution.reducesink.new.enabled=true;
+
+-- 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 vectortab2korc(
+            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)
+STORED AS ORC;
+
+INSERT INTO TABLE vectortab2korc SELECT * FROM vectortab2k;
+
+explain
+select b from vectortab2korc order by b;
+
+select b from vectortab2korc order by b;

http://git-wip-us.apache.org/repos/asf/hive/blob/409db57d/ql/src/test/queries/clientpositive/vector_reduce2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/vector_reduce2.q b/ql/src/test/queries/clientpositive/vector_reduce2.q
new file mode 100644
index 0000000..e8607e2
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/vector_reduce2.q
@@ -0,0 +1,47 @@
+set hive.explain.user=false;
+SET hive.vectorized.execution.enabled=true;
+SET hive.vectorized.execution.reducesink.new.enabled=true;
+
+-- 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 vectortab2korc(
+            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)
+STORED AS ORC;
+
+INSERT INTO TABLE vectortab2korc SELECT * FROM vectortab2k;
+
+explain
+select s, i, s2 from vectortab2korc order by s, i, s2;
+
+select s, i, s2 from vectortab2korc order by s, i, s2;

http://git-wip-us.apache.org/repos/asf/hive/blob/409db57d/ql/src/test/queries/clientpositive/vector_reduce3.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/vector_reduce3.q b/ql/src/test/queries/clientpositive/vector_reduce3.q
new file mode 100644
index 0000000..c0c4288
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/vector_reduce3.q
@@ -0,0 +1,47 @@
+set hive.explain.user=false;
+SET hive.vectorized.execution.enabled=true;
+SET hive.vectorized.execution.reducesink.new.enabled=true;
+
+-- 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 vectortab2korc(
+            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)
+STORED AS ORC;
+
+INSERT INTO TABLE vectortab2korc SELECT * FROM vectortab2k;
+
+explain
+select s from vectortab2korc order by s;
+
+select s from vectortab2korc order by s;

http://git-wip-us.apache.org/repos/asf/hive/blob/409db57d/ql/src/test/results/clientpositive/tez/vector_aggregate_without_gby.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/vector_aggregate_without_gby.q.out b/ql/src/test/results/clientpositive/tez/vector_aggregate_without_gby.q.out
new file mode 100644
index 0000000..9718871
--- /dev/null
+++ b/ql/src/test/results/clientpositive/tez/vector_aggregate_without_gby.q.out
@@ -0,0 +1,85 @@
+PREHOOK: query: create table testvec(id int, dt int, greg_dt string) stored as orc
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@testvec
+POSTHOOK: query: create table testvec(id int, dt int, greg_dt string) stored as orc
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@testvec
+PREHOOK: query: insert into table testvec
+values 
+(1,20150330, '2015-03-30'),
+(2,20150301, '2015-03-01'),
+(3,20150502, '2015-05-02'),
+(4,20150401, '2015-04-01'),
+(5,20150313, '2015-03-13'),
+(6,20150314, '2015-03-14'),
+(7,20150404, '2015-04-04')
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__1
+PREHOOK: Output: default@testvec
+POSTHOOK: query: insert into table testvec
+values 
+(1,20150330, '2015-03-30'),
+(2,20150301, '2015-03-01'),
+(3,20150502, '2015-05-02'),
+(4,20150401, '2015-04-01'),
+(5,20150313, '2015-03-13'),
+(6,20150314, '2015-03-14'),
+(7,20150404, '2015-04-04')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__1
+POSTHOOK: Output: default@testvec
+POSTHOOK: Lineage: testvec.dt EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+POSTHOOK: Lineage: testvec.greg_dt SIMPLE [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col3, type:string, comment:), ]
+POSTHOOK: Lineage: testvec.id EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+PREHOOK: query: explain select max(dt), max(greg_dt) from testvec where id=5
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select max(dt), max(greg_dt) from testvec where id=5
+POSTHOOK: type: QUERY
+Plan optimized by CBO.
+
+Vertex dependency in root stage
+Reducer 2 <- Map 1 (SIMPLE_EDGE)
+
+Stage-0
+   Fetch Operator
+      limit:-1
+      Stage-1
+         Reducer 2
+         File Output Operator [FS_7]
+            compressed:false
+            Statistics:Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: NONE
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
+            Group By Operator [OP_12]
+            |  aggregations:["max(VALUE._col0)","max(VALUE._col1)"]
+            |  outputColumnNames:["_col0","_col1"]
+            |  Statistics:Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: NONE
+            |<-Map 1 [SIMPLE_EDGE]
+               Reduce Output Operator [RS_4]
+                  sort order:
+                  Statistics:Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: NONE
+                  value expressions:_col0 (type: int), _col1 (type: string)
+                  Group By Operator [OP_11]
+                     aggregations:["max(dt)","max(greg_dt)"]
+                     outputColumnNames:["_col0","_col1"]
+                     Statistics:Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: NONE
+                     Select Operator [OP_10]
+                        outputColumnNames:["dt","greg_dt"]
+                        Statistics:Num rows: 3 Data size: 306 Basic stats: COMPLETE Column stats: NONE
+                        Filter Operator [FIL_9]
+                           predicate:(id = 5) (type: boolean)
+                           Statistics:Num rows: 3 Data size: 306 Basic stats: COMPLETE Column stats: NONE
+                           TableScan [TS_0]
+                              alias:testvec
+                              Statistics:Num rows: 7 Data size: 714 Basic stats: COMPLETE Column stats: NONE
+
+PREHOOK: query: select max(dt), max(greg_dt) from testvec where id=5
+PREHOOK: type: QUERY
+PREHOOK: Input: default@testvec
+#### A masked pattern was here ####
+POSTHOOK: query: select max(dt), max(greg_dt) from testvec where id=5
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@testvec
+#### A masked pattern was here ####
+20150313	2015-03-13

http://git-wip-us.apache.org/repos/asf/hive/blob/409db57d/ql/src/test/results/clientpositive/tez/vector_bround.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/vector_bround.q.out b/ql/src/test/results/clientpositive/tez/vector_bround.q.out
new file mode 100644
index 0000000..b520006
--- /dev/null
+++ b/ql/src/test/results/clientpositive/tez/vector_bround.q.out
@@ -0,0 +1,66 @@
+PREHOOK: query: create table test_vector_bround(v0 double, v1 double) stored as orc
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@test_vector_bround
+POSTHOOK: query: create table test_vector_bround(v0 double, v1 double) stored as orc
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@test_vector_bround
+PREHOOK: query: insert into table test_vector_bround
+values
+(2.5, 1.25),
+(3.5, 1.35),
+(-2.5, -1.25),
+(-3.5, -1.35),
+(2.49, 1.249),
+(3.49, 1.349),
+(2.51, 1.251),
+(3.51, 1.351)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__1
+PREHOOK: Output: default@test_vector_bround
+POSTHOOK: query: insert into table test_vector_bround
+values
+(2.5, 1.25),
+(3.5, 1.35),
+(-2.5, -1.25),
+(-3.5, -1.35),
+(2.49, 1.249),
+(3.49, 1.349),
+(2.51, 1.251),
+(3.51, 1.351)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__1
+POSTHOOK: Output: default@test_vector_bround
+POSTHOOK: Lineage: test_vector_bround.v0 EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+POSTHOOK: Lineage: test_vector_bround.v1 EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+PREHOOK: query: explain select bround(v0), bround(v1, 1) from test_vector_bround
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select bround(v0), bround(v1, 1) from test_vector_bround
+POSTHOOK: type: QUERY
+Plan optimized by CBO.
+
+Stage-0
+   Fetch Operator
+      limit:-1
+      Select Operator [SEL_1]
+         outputColumnNames:["_col0","_col1"]
+         TableScan [TS_0]
+            alias:test_vector_bround
+
+PREHOOK: query: select bround(v0), bround(v1, 1) from test_vector_bround
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test_vector_bround
+#### A masked pattern was here ####
+POSTHOOK: query: select bround(v0), bround(v1, 1) from test_vector_bround
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@test_vector_bround
+#### A masked pattern was here ####
+2.0	1.2
+4.0	1.4
+-2.0	-1.2
+-4.0	-1.4
+2.0	1.2
+3.0	1.3
+3.0	1.3
+4.0	1.4

http://git-wip-us.apache.org/repos/asf/hive/blob/409db57d/ql/src/test/results/clientpositive/tez/vector_nvl.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/vector_nvl.q.out b/ql/src/test/results/clientpositive/tez/vector_nvl.q.out
new file mode 100644
index 0000000..b3f83ce
--- /dev/null
+++ b/ql/src/test/results/clientpositive/tez/vector_nvl.q.out
@@ -0,0 +1,194 @@
+PREHOOK: query: EXPLAIN SELECT cdouble, nvl(cdouble, 100) as n
+FROM alltypesorc
+WHERE (cdouble IS NULL)
+LIMIT 10
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN SELECT cdouble, nvl(cdouble, 100) as n
+FROM alltypesorc
+WHERE (cdouble IS NULL)
+LIMIT 10
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: 10
+      Processor Tree:
+        TableScan
+          alias: alltypesorc
+          Filter Operator
+            predicate: cdouble is null (type: boolean)
+            Select Operator
+              expressions: null (type: double), 100.0 (type: double)
+              outputColumnNames: _col0, _col1
+              Limit
+                Number of rows: 10
+                ListSink
+
+PREHOOK: query: SELECT cdouble, nvl(cdouble, 100) as n
+FROM alltypesorc
+WHERE (cdouble IS NULL)
+LIMIT 10
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT cdouble, nvl(cdouble, 100) as n
+FROM alltypesorc
+WHERE (cdouble IS NULL)
+LIMIT 10
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+NULL	100.0
+NULL	100.0
+NULL	100.0
+NULL	100.0
+NULL	100.0
+NULL	100.0
+NULL	100.0
+NULL	100.0
+NULL	100.0
+NULL	100.0
+PREHOOK: query: EXPLAIN SELECT cfloat, nvl(cfloat, 1) as n
+FROM alltypesorc
+LIMIT 10
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN SELECT cfloat, nvl(cfloat, 1) as n
+FROM alltypesorc
+LIMIT 10
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: 10
+      Processor Tree:
+        TableScan
+          alias: alltypesorc
+          Select Operator
+            expressions: cfloat (type: float), if cfloat is null returns1 (type: float)
+            outputColumnNames: _col0, _col1
+            Limit
+              Number of rows: 10
+              ListSink
+
+PREHOOK: query: SELECT cfloat, nvl(cfloat, 1) as n
+FROM alltypesorc
+LIMIT 10
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT cfloat, nvl(cfloat, 1) as n
+FROM alltypesorc
+LIMIT 10
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+-50.0	-50.0
+NULL	1.0
+-28.0	-28.0
+31.0	31.0
+-34.0	-34.0
+29.0	29.0
+31.0	31.0
+27.0	27.0
+-11.0	-11.0
+61.0	61.0
+PREHOOK: query: EXPLAIN SELECT nvl(null, 10) as n
+FROM alltypesorc
+LIMIT 10
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN SELECT nvl(null, 10) as n
+FROM alltypesorc
+LIMIT 10
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: 10
+      Processor Tree:
+        TableScan
+          alias: alltypesorc
+          Select Operator
+            expressions: 10 (type: int)
+            outputColumnNames: _col0
+            Limit
+              Number of rows: 10
+              ListSink
+
+PREHOOK: query: SELECT nvl(null, 10) as n
+FROM alltypesorc
+LIMIT 10
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT nvl(null, 10) as n
+FROM alltypesorc
+LIMIT 10
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+10
+10
+10
+10
+10
+10
+10
+10
+10
+10
+PREHOOK: query: EXPLAIN SELECT nvl(null, null) as n
+FROM alltypesorc
+LIMIT 10
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN SELECT nvl(null, null) as n
+FROM alltypesorc
+LIMIT 10
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: 10
+      Processor Tree:
+        TableScan
+          alias: alltypesorc
+          Select Operator
+            expressions: null (type: void)
+            outputColumnNames: _col0
+            Limit
+              Number of rows: 10
+              ListSink
+
+PREHOOK: query: SELECT nvl(null, null) as n
+FROM alltypesorc
+LIMIT 10
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT nvl(null, null) as n
+FROM alltypesorc
+LIMIT 10
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL

http://git-wip-us.apache.org/repos/asf/hive/blob/409db57d/ql/src/test/results/clientpositive/tez/vector_reduce1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/vector_reduce1.q.out b/ql/src/test/results/clientpositive/tez/vector_reduce1.q.out
new file mode 100644
index 0000000..6035582
--- /dev/null
+++ b/ql/src/test/results/clientpositive/tez/vector_reduce1.q.out
@@ -0,0 +1,2167 @@
+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 vectortab2korc(
+            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)
+STORED AS ORC
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@vectortab2korc
+POSTHOOK: query: create table vectortab2korc(
+            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)
+STORED AS ORC
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@vectortab2korc
+PREHOOK: query: INSERT INTO TABLE vectortab2korc SELECT * FROM vectortab2k
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2k
+PREHOOK: Output: default@vectortab2korc
+POSTHOOK: query: INSERT INTO TABLE vectortab2korc SELECT * FROM vectortab2k
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2k
+POSTHOOK: Output: default@vectortab2korc
+POSTHOOK: Lineage: vectortab2korc.b SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: vectortab2korc.bo SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: vectortab2korc.d SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: vectortab2korc.dc SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:dc, type:decimal(38,18), comment:null), ]
+POSTHOOK: Lineage: vectortab2korc.dt SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:dt, type:date, comment:null), ]
+POSTHOOK: Lineage: vectortab2korc.f SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: vectortab2korc.i SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: vectortab2korc.s SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: vectortab2korc.s2 SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:s2, type:string, comment:null), ]
+POSTHOOK: Lineage: vectortab2korc.si SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: vectortab2korc.t SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:t, type:tinyint, comment:null), ]
+POSTHOOK: Lineage: vectortab2korc.ts SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: vectortab2korc.ts2 SIMPLE [(vectortab2k)vectortab2k.FieldSchema(name:ts2, type:timestamp, comment:null), ]
+PREHOOK: query: explain
+select b from vectortab2korc order by b
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select b from vectortab2korc order by b
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+      Edges:
+        Reducer 2 <- Map 1 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: vectortab2korc
+                  Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: b (type: bigint)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: _col0 (type: bigint)
+                      sort order: +
+                      Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE
+            Execution mode: vectorized
+        Reducer 2 
+            Execution mode: vectorized
+            Reduce Operator Tree:
+              Select Operator
+                expressions: KEY.reducesinkkey0 (type: bigint)
+                outputColumnNames: _col0
+                Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select b from vectortab2korc order by b
+PREHOOK: type: QUERY
+PREHOOK: Input: default@vectortab2korc
+#### A masked pattern was here ####
+POSTHOOK: query: select b from vectortab2korc order by b
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@vectortab2korc
+#### A masked pattern was here ####
+-6917607783359897600
+-6919476845891313664
+-6920172215209426944
+-6921654334727036928
+-6933565857643814912
+-6934304742087655424
+-6935038507792801792
+-6935548339131138048
+-6938706403992854528
+-6941777546186579968
+-6947955278050181120
+-6951350560260784128
+-6957946688477274112
+-6960947572095770624
+-6962271229404348416
+-6962292590214234112
+-6968771079156654080
+-6968892545529896960
+-6970396058557005824
+-6974654664348033024
+-6975459232300236800
+-6986178228432322560
+-6988811476286873600
+-6988970700649168896
+-6992217501957169152
+-6997233584896229376
+-7000925438663041024
+-7003696402314215424
+-7011425384222244864
+-7017212700635545600
+-7020852530219171840
+-7030489936116252672
+-7035132060308643840
+-7036607470351654912
+-7037375807670501376
+-7037638331316469760
+-7038455462786334720
+-7040248820505149440
+-7041362811802148864
+-7042183597114081280
+-7046180371529351168
+-7049618574399692800
+-7052619594823221248
+-7055619148037554176
+-7055760785575665664
+-7057750467944931328
+-7058986555327307776
+-7063777488249085952
+-7078068944081002496
+-7079898537463537664
+-7081500255163727872
+-7083646746411720704
+-7085247548404178944
+-7093825013581979648
+-7094189393339678720
+-7094827141662539776
+-7104310188119834624
+-7106210529681350656
+-7109790267244814336
+-7115054815375073280
+-7120456708338688000
+-7127548949860818944
+-7138415011665043456
+-7139677575412686848
+-7140008543769042944
+-7144791190333546496
+-7145585429014888448
+-7147490721376591872
+-7152177800841502720
+-7155539549555105792
+-7158472098920390656
+-7159700138947862528
+-7161165959057334272
+-7162299524557471744
+-7172594404186693632
+-7185369278665605120
+-7192529627893858304
+-7194281951646187520
+-7195217207163166720
+-7198372044947275776
+-7199983995864711168
+-7201085131997011968
+-7209060152494817280
+-7213775605408178176
+-7220731681653604352
+-7221474017515347968
+-7228589258642194432
+-7240213957902663680
+-7242345057866285056
+-7245872320493322240
+-7246123871306244096
+-7255010240787030016
+-7255686273677328384
+-7262049693594943488
+-7262384251828518912
+-7262798781688651776
+-7263060340185194496
+-7265998318110711808
+-7266719102957125632
+-7270034223527993344
+-7273590251991162880
+-7273694358642851840
+-7276111129363046400
+-7287583262310350848
+-7292078334519894016
+-7296096276653391872
+-7303847963918393344
+-7319315187617587200
+-7326863346317598720
+-7328087811698909184
+-7329767178250018816
+-7329807949048193024
+-7330203470474985472
+-7330413050756235264
+-7333278178640953344
+-7333362172439035904
+-7340231535789727744
+-7344146703223496704
+-7344947507044466688
+-7345562788132315136
+-7356685674003021824
+-7357888618985873408
+-7362189611124563968
+-7366430883634929664
+-7378096180613840896
+-7380731416973295616
+-7395343938785738752
+-7395553021620731904
+-7399631791131074560
+-7404052043914526720
+-7404057145074712576
+-7409317158045442048
+-7409653086454030336
+-7412431471807283200
+-7413317118463164416
+-7419068456205385728
+-7420448501073051648
+-7425160895830573056
+-7429331808102899712
+-7433265617153343488
+-7442593976514420736
+-7444070205513138176
+-7451660755269853184
+-7453525026342617088
+-7455898404374921216
+-7456869587112255488
+-7461750143936897024
+-7464270453557993472
+-7469660864676585472
+-7470307155642245120
+-7476082621253402624
+-7483435388852559872
+-7488345684795342848
+-7488415863027367936
+-7494411162675691520
+-7496839341561954304
+-7497303453253402624
+-7500200359698907136
+-7501803640821456896
+-7506254246954500096
+-7507424948896415744
+-7507578199583694848
+-7510418793070075904
+-7511202710200885248
+-7511952204985049088
+-7512289590991544320
+-7512297136103800832
+-7515996202498473984
+-7524170566881329152
+-7526793959592140800
+-7528526815026692096
+-7532751268425261056
+-7535857766791577600
+-7535958203887706112
+-7536330682873937920
+-7540104552219860992
+-7541860097718902784
+-7542857121910046720
+-7547245548870025216
+-7547432761381339136
+-7551394356730339328
+-7557017910095650816
+-7558524160894427136
+-7571293705217687552
+-7571957778022178816
+-7572262898020278272
+-7572962089372991488
+-7576194692683563008
+-7593363318079610880
+-7594824008626372608
+-7598782894648565760
+-7600138468036386816
+-7603467428164009984
+-7603569103205916672
+-7610137349734883328
+-7611584069753552896
+-7612455481940246528
+-7612466483992051712
+-7616522969329262592
+-7617860842651017216
+-7623047151287754752
+-7623359796281999360
+-7623405558242500608
+-7624057992767782912
+-7629401308029976576
+-7637494527844343808
+-7637755520917741568
+-7642381493746483200
+-7647020450676146176
+-7661192563533062144
+-7661250850555633664
+-7663293054873812992
+-7665186441284968448
+-7668388017287020544
+-7669169138124275712
+-7673901622181953536
+-7679894005808693248
+-7686220526274502656
+-7687052294777208832
+-7692192232238678016
+-7695491171376291840
+-7700203302632210432
+-7703540456272994304
+-7707242953271500800
+-7707867749256445952
+-7708932208121225216
+-7709958788604936192
+-7712425776235274240
+-7720966287634112512
+-7739424919198187520
+-7744462446680375296
+-7751265769984491520
+-7751427073017544704
+-7753051494275432448
+-7759238919361888256
+-7759425383684849664
+-7772064021830574080
+-7773957003968675840
+-7777884099756122112
+-7778829032042790912
+-7779270198785875968
+-7782344916178796544
+-7784419454650843136
+-7792903881635938304
+-7793447076762345472
+-7797149520019062784
+-7797151404935618560
+-7800879252150779904
+-7802538500225777664
+-7804116532814151680
+-7805985795815342080
+-7811060170911375360
+-7818454479651135488
+-7819437864839495680
+-7822452149325094912
+-7824788571789279232
+-7827420207675105280
+-7831320202242228224
+-7831595638727565312
+-7833618000492109824
+-7835907977757245440
+-7838598833900584960
+-7840338174858199040
+-7845896959112658944
+-7848043121524228096
+-7849504559236210688
+-7858505678035951616
+-7866079955473989632
+-7867219225874571264
+-7868306678534193152
+-7873753603299540992
+-7875953567586451456
+-7877598807023386624
+-7878145001776152576
+-7879864376629567488
+-7881262505761710080
+-7881351200983613440
+-7883252982752665600
+-7884460946615984128
+-7888051992910274560
+-7892780594910871552
+-7893577088764174336
+-7894382303337832448
+-7895991410072928256
+-7902517224300036096
+-7903158849011843072
+-7904188195431661568
+-7907355742053883904
+-7910019233726242816
+-7911421221625077760
+-7915999634274369536
+-7916510129632296960
+-7928062266382778368
+-7928440849566146560
+-7939634346485858304
+-7949309059286163456
+-7949445503604604928
+-7953426740065312768
+-7964801953178091520
+-7966960765508280320
+-7978782649203228672
+-7989766326847807488
+-7998947380180819968
+-8007017894942638080
+-8013397854633648128
+-8016589197379289088
+-8017791189288869888
+-8018511948141748224
+-8021859935185928192
+-8022573309127000064
+-8023708819947323392
+-8028275725610909696
+-8028910243475038208
+-8030058711611629568
+-8034414142083170304
+-8046189486447017984
+-8046238369820344320
+-8047774491688255488
+-8051395538179063808
+-8051587217208967168
+-8051871680800120832
+-8054581198284668928
+-8067243114610532352
+-8070535484085895168
+-8076479329071955968
+-8082793390939193344
+-8084716955963252736
+-8086577583338061824
+-8088337436168830976
+-8099313480512716800
+-8103788088118018048
+-8104684579106914304
+-8108693586698706944
+-8115963579415650304
+-8117838333114212352
+-8122639684164501504
+-8127494999848919040
+-8131997716860526592
+-8136227554401107968
+-8140349174954893312
+-8142667274351345664
+-8147405381260345344
+-8158011642485825536
+-8161047750470279168
+-8172827216441573376
+-8182421179156905984
+-8191825921746305024
+-8194062064124362752
+-8203008052020879360
+-8203075743525806080
+-8205148279289085952
+-8214462866994339840
+-8219876839318716416
+-8232763638546694144
+-8240034910581153792
+-8240684139569233920
+-8243487285852766208
+-8244116388227104768
+-8244657976255889408
+-8260340354454503424
+-8269917980278980608
+-8270479187688816640
+-8275337702906757120
+-8280276629934981120
+-8293833565967810560
+-8297230235506343936
+-8300526097982226432
+-8300764106868350976
+-8302817097848307712
+-8317591428117274624
+-8318886086186213376
+-8322751250650218496
+-8330233444291084288
+-8335810316927213568
+-8340523561480437760
+-8345065519816695808
+-8347088645602050048
+-8357136656913686528
+-8358130693961195520
+-8359839265974165504
+-8368269352975982592
+-8368487814665895936
+-8369487968903897088
+-8379109122834997248
+-8379964450833367040
+-8384695077413412864
+-8387347109404286976
+-8387536830476820480
+-8395998375405912064
+-8400045653258444800
+-8411282676082565120
+-8418913260807217152
+-8425998949410889728
+-8426531414463545344
+-8430283518005846016
+-8430370933326536704
+-8431492599012163584
+-8438554249514491904
+-8445801063348281344
+-8453491903284994048
+-8454143651040444416
+-8465978403747037184
+-8469607298426437632
+-8471480409335513088
+-8485389240529354752
+-8488247955875618816
+-8490382417169408000
+-8494118409594650624
+-8503342882470019072
+-8503573595507761152
+-8507279516485566464
+-8509547439040757760
+-8518060755719585792
+-8518258741831680000
+-8521578237232529408
+-8522878384019169280
+-8523434203900674048
+-8525212657458348032
+-8535957064499879936
+-8536369662934401024
+-8543982423727128576
+-8544299740525461504
+-8545239748068941824
+-8546758906409312256
+-8552393882631389184
+-8555709701170552832
+-8559008501282832384
+-8559252110266564608
+-8562524688907485184
+-8566856504746352640
+-8566940231897874432
+-8570933074545745920
+-8572823448513445888
+-8572949572756774912
+-8581765103969312768
+-8581979259158929408
+-8584520406368493568
+-8585134536083660800
+-8585966098173870080
+-8593419958317056000
+-8603817012434198528
+-8604758220106014720
+-8607195685207408640
+-8615168537390571520
+-8619303037130301440
+-8623238306523824128
+-8623965248051789824
+-8632237187473088512
+-8649711322250362880
+-8651641150831362048
+-8654433008222797824
+-8654797319350927360
+-8658387566611996672
+-8659643752269242368
+-8659692318743314432
+-8660149447361404928
+-8664374244449050624
+-8664806103426252800
+-8665218198816497664
+-8665764757143658496
+-8675661101615489024
+-8675892979328212992
+-8683802826440105984
+-8688153842294595584
+-8689606130068611072
+-8694818694700048384
+-8696162322976997376
+-8703026916864802816
+-8704234107608203264
+-8705403811649355776
+-8710298418608619520
+-8714995808835444736
+-8719510423723155456
+-8730803262481580032
+-8731068123910987776
+-8746702976270385152
+-8754966081778565120
+-8754992450211692544
+-8756989568739835904
+-8760655406971863040
+-8763062627136864256
+-8768744394742235136
+-8782213262837530624
+-8783777723063099392
+-8789178184387641344
+-8797972842900307968
+-8807361476639629312
+-8813211231120031744
+-8831091081349758976
+-8832750849949892608
+-8833019327569510400
+-8835408234247168000
+-8836899523028312064
+-8843859708698583040
+-8844949406948671488
+-8845239510002753536
+-8852770376039219200
+-8853553406533894144
+-8856151919723003904
+-8856821118526734336
+-8857335871148171264
+-8858063395050110976
+-8859107121649893376
+-8866442231663067136
+-8870186814744420352
+-8870673219965001728
+-8875546987176206336
+-8877053610728161280
+-8877431933441327104
+-8879742387365429248
+-8881446757271846912
+-8887058200926093312
+-8892963883085578240
+-8896045754034978816
+-8914039133569400832
+-8916987977485312000
+-8922409715403112448
+-8923529803981905920
+-8927968289860370432
+-8930307926221807616
+-8938849835283677184
+-8940944155843461120
+-8941201923743703040
+-8946656952763777024
+-8948335470186373120
+-8959796625322680320
+-8961059046745669632
+-8962547695651323904
+-8965578088652095488
+-8989473881707921408
+-8990843030306717696
+-8992599250893979648
+-8996954350906294272
+-9002912355472736256
+-9004892183139811328
+-9008631121684832256
+-9012093603044245504
+-9013952631912325120
+-9014145341570203648
+-9022154842129547264
+-9032650742739836928
+-9049720998034137088
+-9051477157204770816
+-9058029636530003968
+-9066993118333706240
+-9071565764086521856
+-9075302542655684608
+-9075486079396069376
+-9078662294976061440
+-9079801920509001728
+-9080568167841226752
+-9080956291212132352
+-9084940280061485056
+-9088239683374350336
+-9091113592821972992
+-9095689235523264512
+-9101953184875757568
+-9102482277760983040
+-9105358806324035584
+-9105701280936501248
+-9109392978217484288
+-9117959922369060864
+-9126793997498957824
+-9136398397785948160
+-9142610685888192512
+-9145593811310010368
+-9148197394287779840
+-9149719074367946752
+-9157613004431998976
+-9175038118837149696
+-9175279464813223936
+-9178166810751909888
+-9187662685618348032
+-9189155542884474880
+-9203804401302323200
+-9203942396257984512
+-9206329156028112896
+-9210275791460499456
+-9213132862973829120
+-9215144824304721920
+-9218875542187065344
+-9219066990552760320
+1021
+1030
+1032
+1039
+1046
+1048
+1053
+1055
+1058
+1065
+1066
+1074
+1075
+1075
+1075
+108
+1086
+1093
+1094
+1095
+1099
+1115
+112
+1127
+1128
+1132
+1134
+1141
+1142
+1145
+1153
+1157
+1158
+1165
+1165
+1168
+1177
+1187
+1189
+1198
+120
+1201
+1217
+1234
+1243
+1247
+1252
+1261
+1270
+1280
+1282
+1286
+1287
+1290
+1291
+1299
+130
+1307
+1312
+1316
+1321
+1337
+1341
+1342
+1343
+1345
+1346
+135
+1366
+1368
+1368
+1371
+1371
+138
+1386
+1398
+1409
+1422
+1423
+1436
+1439
+1447
+1450
+1454
+1458
+1462
+1466
+1470
+1477
+1481
+1481
+1489
+1493
+1495
+1501
+1506
+1508
+1509
+1509
+1518
+1520
+1521
+1524
+1530
+1537
+1537
+154
+154
+1541
+1542
+1545
+1556
+1559
+1561
+1566
+1604
+1606
+1608
+1613
+1614
+1620
+1638
+1641
+1643
+1648
+1651
+1667
+1671
+1674
+1676
+1678
+168
+1681
+169
+1693
+1701
+1701
+1704
+1719
+1719
+1726
+1728
+1745
+1751
+1752
+1769
+1774
+1775
+1777
+1777
+1780
+1781
+1785
+1786
+1788
+1789
+1791
+1796
+1806
+181
+1811
+1813
+1826
+1827
+1835
+1837
+1845
+1846
+1856
+1856
+1862
+1863
+1864
+1866
+187
+1870
+188
+1880
+1890
+1892
+1899
+19
+19
+1906
+1910
+1914
+1914
+1926
+1937
+1940
+1941
+1948
+1948
+1948
+1955
+1965
+1972
+1981
+1983
+1987
+1990
+1995
+1999
+2001
+2002
+2004
+2009
+2011
+2013
+2016
+2017
+2020
+2020
+2025
+2026
+2029
+203
+204
+2046
+2056
+2067
+2072
+2073
+2085
+2089
+2092
+2105
+2106
+2108
+213
+213
+2131
+2138
+2140
+2144
+2155
+2177
+2179
+2180
+2183
+2186
+2187
+2189
+2193
+2193
+2194
+22
+2201
+2205
+2214
+2217
+2218
+2223
+2227
+2229
+2232
+2241
+2244
+2255
+2262
+2264
+2270
+2274
+2277
+2279
+228
+2283
+2285
+2285
+2295
+2306
+2320
+2323
+2325
+2325
+2335
+2341
+2348
+2358
+236
+2373
+238
+2386
+2393
+2393
+2398
+2400
+2410
+2412
+2412
+2420
+2426
+2434
+244
+2461
+2463
+2463
+2463
+2465
+2469
+2475
+2476
+2485
+2485
+2487
+2492
+2494
+2502
+2506
+2509
+2512
+2514
+2515
+2517
+2524
+2533
+2539
+2540
+255
+2551
+2553
+2560
+2560
+2563
+2565
+2569
+2579
+2580
+2587
+259
+2599
+2607
+2608
+2619
+2619
+2625
+2626
+263
+263
+2637
+2647
+2649
+2662
+2663
+2675
+268
+268
+2680
+2682
+2688
+2689
+2692
+2700
+2712
+2714
+2715
+2715
+2719
+2724
+2725
+2735
+2745
+275
+2752
+2762
+2772
+2776
+2786
+2786
+279
+2790
+2791
+2803
+2803
+2803
+2805
+281
+2810
+2811
+2816
+2821
+2824
+2835
+2842
+2843
+2843
+2846
+2847
+2848
+2850
+2855
+2855
+2862
+2878
+2886
+289
+2897
+2897
+2900
+2903
+2905
+2911
+2915
+2919
+2933
+2933
+2938
+294
+2941
+2942
+296
+296
+2962
+2968
+2968
+2971
+2977
+2979
+2984
+2986
+2988
+2991
+3002
+3006
+301
+302
+3021
+3021
+3024
+3029
+3031
+3036
+3043
+3054
+3055
+3058
+3059
+3060
+3060
+3067
+3071
+3073
+3079
+3079
+3083
+3084
+3089
+3094
+3103
+311
+3111
+3118
+3119
+3144
+3147
+3159
+3159
+3163
+3174
+3183
+3190
+3197
+3199
+320
+3203
+3206
+3208
+3212
+3213
+3231
+3232
+3235
+3244
+3245
+3248
+3249
+3253
+3255
+3263
+3286
+3300
+3307
+3322
+3333
+3352
+336
+3365
+3366
+3397
+34
+3401
+3407
+3409
+341
+3418
+3418
+342
+3421
+3430
+3443
+3446
+345
+3456
+346
+346
+3460
+3462
+3462
+3462
+3467
+3467
+347
+3472
+3478
+3493
+350
+3507
+3510
+3512
+3533
+3534
+3541
+3542
+355
+3554
+3555
+3555
+3563
+3566
+3567
+3568
+3579
+3588
+3588
+3599
+3606
+3608
+3609
+361
+3613
+3622
+3622
+3625
+3630
+3637
+364
+3648
+3663
+3664
+367
+3672
+3673
+3677
+3680
+3682
+3690
+3691
+3701
+3702
+3703
+3707
+3722
+3724
+3725
+3725
+3728
+3728
+3739
+3747
+3749
+375
+3755
+3763
+3764
+3769
+3770
+3770
+378
+3781
+3781
+3789
+379
+3810
+3812
+3823
+3824
+383
+383
+3830
+3835
+3841
+3848
+3858
+3860
+3866
+3866
+3874
+3879
+388
+3887
+3901
+3904
+3907
+391
+3910
+3911
+3913
+392
+3932
+3940
+3941
+3945
+3946
+3949
+3958
+3960
+3961
+3962
+3965
+3974
+3974
+3980
+3990
+4018
+4020
+4024
+4030
+4037
+4051
+4054
+4056
+4075
+4078
+4088
+41
+412
+412
+417
+425
+443
+454
+455
+462
+470
+471
+481
+482
+485
+489
+49
+490
+491
+5
+500
+501
+501
+504
+522
+523
+524
+530
+535
+579
+583
+584
+586
+587
+590
+597
+601
+612
+615
+618
+65
+650
+658
+66
+661
+661
+663
+664
+677
+68
+681
+687
+688
+690
+691
+6923604860394528768
+6924820982050758656
+6926925215281774592
+6927260280037097472
+6928080429732536320
+6933001829416034304
+6933451028794925056
+6933731240564056064
+6934570741217755136
+694
+6947488599548215296
+695
+6960137166475911168
+6962726713896484864
+6963217546192322560
+6964585306125008896
+6967631925774639104
+6969599299897163776
+6974475559697768448
+6982145326341423104
+6987889924212203520
+6991316084916879360
+6996686091335884800
+7006803044329021440
+7013693841855774720
+7014537632150224896
+7017956982081404928
+7022349041913978880
+7027529814236192768
+7031339012080549888
+7039820685967343616
+7045967493826387968
+7049773031131283456
+7052226236896256000
+7054271419461812224
+7054938591408996352
+7060236714847412224
+7061498706968428544
+7061809776248545280
+7062382339142156288
+7062605127422894080
+7065344324692443136
+7068517339681259520
+7069729473166090240
+707
+7077311975029555200
+7078641038157643776
+7080269176324218880
+7084659344078970880
+7086206629592252416
+7091300332052062208
+7099005292698550272
+71
+7107604675626008576
+7125231541858205696
+7128222874437238784
+7130159794259353600
+7130306447560826880
+7149417430082027520
+7153922334283776000
+7157247449513484288
+7164349895861829632
+7165364563962191872
+7166263463731421184
+7175638927948562432
+7186401810812059648
+7195454019231834112
+7198687580227043328
+7199539820886958080
+7204802700490858496
+7210160489915236352
+7212016545671348224
+7212090742612467712
+7217123582035116032
+7220131672176058368
+7220581538170413056
+7223569671814987776
+7226360892091416576
+7229607057201127424
+723
+7231399302953377792
+7232273749940838400
+7235109456886816768
+7237310132329488384
+7238339720750948352
+724
+7242751359672631296
+7249443195032985600
+7250237407877382144
+7254710367022645248
+7255302164215013376
+7259955893466931200
+7260908278294560768
+7265141874315517952
+7266437490436341760
+7271786885641666560
+7271887863395459072
+7274777328897802240
+7291432593139507200
+7295502697317097472
+7295926343524163584
+7296164580491075584
+7299197687217856512
+73
+7304839835188609024
+7308289763456000000
+7309156463509061632
+7310869618402910208
+7319711402123149312
+7333512171174223872
+7339426767877390336
+7343171468838567936
+7344029858387820544
+7345991518378442752
+7347732772348870656
+7348598907182800896
+735
+7354813692542304256
+7359004378440146944
+736
+7368920486374989824
+7370078518278397952
+7370803940448305152
+7375521127126089728
+7376467688511455232
+7378993334503694336
+738
+7381659098423926784
+7384150968511315968
+7386087924003676160
+7391208370547269632
+7393308503950548992
+7394967727502467072
+7401968422230032384
+7410096605330227200
+7410872053689794560
+7411793502161182720
+7412924364686458880
+7414865343000322048
+7418271723644403712
+743
+7432428551399669760
+7432998950057975808
+7436133434239229952
+7440265908266827776
+7450416810848313344
+7452756603516190720
+7454442625055145984
+7454632396542074880
+7461153404961128448
+7471208109437304832
+7473537548003352576
+7486884806277611520
+7487338208419823616
+7487538600082554880
+7490717730239250432
+7491898395977523200
+7492436934952574976
+7497276415392407552
+7497306924248834048
+7500716020874674176
+7514552840617558016
+7517159036469575680
+7524958388842078208
+7528074274555305984
+7528211148397944832
+7534042483076857856
+7534145866886782976
+7534549597202194432
+7545689659010949120
+7548958830580563968
+7549858023389003776
+7555301305375858688
+7566273236152721408
+7569249672628789248
+7570474972934488064
+7573530789362262016
+7575087487730196480
+7581052107944361984
+7581614118458335232
+7584007864107778048
+7592440105065308160
+7593521922173419520
+7596563216912211968
+7599019810193211392
+7608447395949109248
+7614435638888210432
+7620183559667081216
+7621013099259527168
+7625728883085025280
+7626715182847090688
+763
+7637152193832886272
+7647481735646363648
+7648729477297987584
+7652123583449161728
+7659279803863146496
+7662037650719850496
+7675009476762918912
+7678790769408172032
+7682327310082531328
+7686992843032010752
+7689489436826804224
+7690986322714066944
+7691062622443044864
+7696737688942567424
+7697541332524376064
+7700734109530767360
+7701723309715685376
+7705445437881278464
+7710447533880614912
+7718825401976684544
+7720187583697502208
+7731443941834678272
+7735566678126616576
+774
+7741854854673367040
+7746402369011277824
+7747874976739016704
+7748799008146366464
+7752740515534422016
+7753359568986636288
+7753882935005880320
+7761834341179375616
+7762823913046556672
+7765456790394871808
+7768984605670604800
+7775034125776363520
+7778936842502275072
+7779486624537370624
+7779735136559579136
+7782245855193874432
+7784169796350730240
+7784489776013295616
+779
+7790728456522784768
+7792036342592348160
+7794244032613703680
+78
+780
+7800332581637259264
+7801697837312884736
+7818464507324121088
+782
+7823874904139849728
+784
+7843804446688264192
+7844258063629852672
+7845953007588401152
+7857878068300898304
+7868367829080506368
+7870277756614623232
+7871189141676998656
+7871554728617025536
+7874764415950176256
+7885697257930588160
+7888238729321496576
+789
+7892026679115554816
+7892281003266408448
+7898670840507031552
+7909645665163804672
+7917494645725765632
+7919597361814577152
+7921639119138070528
+7922443154272395264
+7926898770090491904
+7933040277013962752
+7936149988210212864
+7944741547145502720
+7947544013461512192
+7948803266578161664
+7955126053367119872
+7961515985722605568
+7961909238130270208
+797
+7983789401706094592
+7989119273552158720
+7989160253372817408
+7997694023324975104
+7998357471114969088
+7998687089080467456
+80
+8000440057238052864
+8002769767000145920
+8004633750273925120
+8011181697250631680
+8011602724663336960
+8014986215157530624
+8017403886247927808
+803
+8045070943673671680
+8048726769133592576
+8059284960252731392
+8069531888205086720
+8071961599867387904
+8073733016154431488
+8079573715140485120
+808
+8087737899452432384
+809
+8091421389575282688
+8099215208813903872
+8100036735858401280
+8109381965028548608
+8111757081791733760
+8113585123802529792
+8116738401948377088
+812
+8120593157178228736
+8129551357032259584
+8135164922674872320
+8142241016679735296
+8143462899383345152
+8144552446127972352
+8145745969573666816
+8145750910080745472
+8146288732715196416
+8146492373537660928
+8148211378319933440
+815
+8150115791664340992
+8156018594610790400
+8156782979767238656
+8160569434550403072
+8160662610166194176
+8163948965373386752
+8168742078705262592
+8169878743136043008
+8171188598958407680
+8183233196086214656
+8184799300477943808
+8190539859890601984
+8190967051000659968
+8192304692696383488
+8195103847607967744
+8199513544090730496
+820
+820
+8201303040648052736
+8201491077550874624
+8208354137450766336
+8210813831744118784
+8213810702473183232
+8219326436390821888
+8220104397160169472
+8221561626658881536
+8222714144797368320
+8223732800007864320
+823
+8230371298967609344
+8235179243092090880
+8244041599171862528
+8254763178969915392
+8268875586442256384
+8269730157217062912
+8272001752345690112
+8279056098670198784
+8282648443538710528
+8283099811330506752
+8286706213485297664
+8287522765741301760
+8290014929764040704
+8290944180915871744
+8294315622451740672
+8295110846998233088
+83
+8302473563519950848
+8316336224427483136
+8323460620425330688
+8325227661920133120
+8332670681629106176
+8333523087360901120
+8337549596011102208
+8345435427356090368
+835
+8351163199364390912
+8362046808797306880
+8365058996333953024
+8367680396909404160
+8368012468775608320
+837
+8371939471056470016
+8372408423196270592
+8372588378498777088
+8374321007870836736
+8376440110255243264
+8383159090746204160
+8388363436324085760
+8391407951622815744
+8391785334471589888
+8396433451610652672
+8398862954249560064
+8407869317250220032
+8410599906334097408
+8411494452500930560
+8415171956168417280
+8416121695917498368
+8417381121663746048
+8419958579638157312
+8424515140664360960
+8435912708683087872
+845
+8451612303224520704
+8454154705460666368
+8455496814886002688
+8457906374051020800
+8461498293348065280
+8463868417649524736
+8467976965865799680
+8470141334513098752
+8472429318602268672
+8473699639908261888
+8487573502287478784
+8489584373231919104
+8489735221193138176
+85
+8501910015960735744
+8508401924853850112
+8509508263705477120
+8514851182589771776
+8514979402185596928
+8515682078777081856
+8518454006987948032
+8519937082746634240
+8523972434954510336
+8524940073536954368
+8525336514806317056
+8525894870444638208
+8532016240026279936
+8536948829863198720
+8540237852367446016
+8543177193114779648
+8547243497773457408
+8551446856960942080
+8553195689344991232
+8554899472487596032
+8555933456197828608
+8555948987770511360
+8557218322962644992
+8558000156325707776
+8560526613401714688
+8569030475428511744
+8570983266408103936
+8571268359622172672
+8573305425181941760
+8577096957495025664
+8579974641030365184
+8583916402383601664
+8613562211893919744
+8625937019655200768
+8631515095562887168
+8637720762289659904
+8639254009546055680
+8641221723991433216
+8643198489997254656
+8644602243484803072
+8649296591032172544
+8652485812846567424
+8656571350884048896
+8660248367767076864
+8665969966920990720
+8666178591503564800
+8677632093825916928
+8677794924343164928
+868
+8682955459667951616
+8687042963221159936
+8688483860094599168
+8693036785094565888
+8697823501349609472
+8698055291501543424
+8708232769657815040
+8708845895460577280
+871
+8714829359200747520
+8716401555586727936
+8720504651219001344
+8723248113030782976
+873
+8731960288562044928
+8734584858442498048
+8736061027343859712
+874
+8752150411997356032
+8759089349412847616
+8759184090543857664
+8760285623204290560
+8761174805938331648
+8769199243315814400
+8773222500321361920
+8775009214012456960
+8779073705407963136
+8779711700787298304
+878
+8780196485890555904
+8782900615468302336
+8783241818558193664
+8785153741735616512
+8792059919353348096
+8793387410919038976
+8795069490394882048
+8806507556248731648
+8808467247666241536
+8811693967537774592
+8815398225009967104
+8817665768680906752
+8822384228057604096
+8825059717746376704
+8829545979081744384
+883
+8836228556823977984
+8837420822750314496
+8849475396952514560
+8850055384477401088
+8853989376829833216
+8854495099223375872
+8854677881758162944
+8854715632851345408
+8856674723376668672
+8868529429494071296
+8871707618793996288
+8875745082589929472
+888
+8895174927321243648
+8896237972875370496
+8897901899039473664
+8899122608190930944
+8900180888218329088
+8900351886974279680
+8900545829211299840
+8905330479248064512
+8910706980937261056
+8920344895701393408
+8920533610804609024
+8927691194719174656
+8928133990107881472
+8935252708196999168
+8936639033158410240
+8939431770838810624
+8945004737083555840
+8945302550165004288
+8962097525980225536
+8972161729142095872
+8979012655944220672
+898
+898
+8983857919580209152
+8983912573761167360
+8984935029383389184
+8987827141270880256
+8991071342495531008
+8991442360387584000
+8994608999945125888
+8995562121346260992
+8996824426131390464
+9000633029632499712
+9001907486943993856
+9005866015985713152
+9016280522993975296
+9020143715350814720
+9023663198045544448
+9030480306789818368
+9038087402564657152
+9040958359122640896
+9043089884440068096
+9048002942653710336
+9048297564833079296
+9050032047355125760
+9053187076403060736
+9054887854393950208
+9062227900376203264
+9064847977742032896
+9067985867711291392
+9073672806863790080
+9075404705968840704
+9078604269481148416
+908
+9083076230151864320
+9083704659251798016
+9084402694981533696
+9085381906890203136
+9085434340468473856
+9086905513121890304
+9089435102788009984
+9091082386452684800
+9091085792947666944
+9094945190752903168
+9096395849845194752
+91
+9104574294205636608
+9107991000536498176
+9112400579327483904
+9114850402293882880
+9116137265342169088
+9117063974299148288
+9119046173224370176
+9123116008004288512
+913
+9131533983989358592
+9132009829414584320
+9136234417125007360
+9136548192574529536
+9139805788041134080
+914
+9148071980848742400
+9149216169284091904
+9165199002069458944
+9169248521377374208
+917
+9174894805640142848
+918
+9180098147855769600
+9182828596851990528
+9185458640237641728
+9185952983951343616
+9188173682239275008
+919
+9190466190353661952
+9191943992860327936
+9194388393453060096
+9199741683232399360
+9207107990561972224
+9207927479837319168
+9209153648361848832
+921
+9211455920344088576
+922
+923
+927
+928
+939
+94
+945
+947
+950
+950
+958
+961
+965
+967
+976
+979
+982
+987
+997
+999
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL