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 2018/05/24 21:41:55 UTC
[1/2] hive git commit: HIVE-19498: Vectorization: CAST expressions
produce wrong results (Matt McCline, reviewed by Teddy Choi)
Repository: hive
Updated Branches:
refs/heads/master 2c848ef5d -> 25aaf7db0
http://git-wip-us.apache.org/repos/asf/hive/blob/25aaf7db/ql/src/test/results/clientpositive/spark/vector_decimal_aggregate.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/vector_decimal_aggregate.q.out b/ql/src/test/results/clientpositive/spark/vector_decimal_aggregate.q.out
index d37a27e..c5d0214 100644
--- a/ql/src/test/results/clientpositive/spark/vector_decimal_aggregate.q.out
+++ b/ql/src/test/results/clientpositive/spark/vector_decimal_aggregate.q.out
@@ -457,7 +457,7 @@ STAGE PLANS:
Map Operator Tree:
TableScan
alias: decimal_vgby_small
- Statistics: Num rows: 12289 Data size: 346472 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 12289 Data size: 346462 Basic stats: COMPLETE Column stats: NONE
TableScan Vectorization:
native: true
vectorizationSchemaColumns: [0:cdouble:double, 1:cdecimal1:decimal(11,5)/DECIMAL_64, 2:cdecimal2:decimal(16,0)/DECIMAL_64, 3:cint:int, 4:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
@@ -468,7 +468,7 @@ STAGE PLANS:
className: VectorSelectOperator
native: true
projectedOutputColumnNums: [1, 2, 3]
- Statistics: Num rows: 12289 Data size: 346472 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 12289 Data size: 346462 Basic stats: COMPLETE Column stats: NONE
Group By Operator
aggregations: count(cdecimal1), max(cdecimal1), min(cdecimal1), sum(cdecimal1), count(cdecimal2), max(cdecimal2), min(cdecimal2), sum(cdecimal2), count()
Group By Vectorization:
@@ -482,7 +482,7 @@ STAGE PLANS:
keys: cint (type: int)
mode: hash
outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
- Statistics: Num rows: 12289 Data size: 346472 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 12289 Data size: 346462 Basic stats: COMPLETE Column stats: NONE
Reduce Output Operator
key expressions: _col0 (type: int)
sort order: +
@@ -493,7 +493,7 @@ STAGE PLANS:
native: true
nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
valueColumnNums: [1, 2, 3, 4, 5, 6, 7, 8, 9]
- Statistics: Num rows: 12289 Data size: 346472 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 12289 Data size: 346462 Basic stats: COMPLETE Column stats: NONE
value expressions: _col1 (type: bigint), _col2 (type: decimal(11,5)), _col3 (type: decimal(11,5)), _col4 (type: decimal(21,5)), _col5 (type: bigint), _col6 (type: decimal(16,0)), _col7 (type: decimal(16,0)), _col8 (type: decimal(26,0)), _col9 (type: bigint)
Execution mode: vectorized
Map Vectorization:
@@ -540,14 +540,14 @@ STAGE PLANS:
keys: KEY._col0 (type: int)
mode: mergepartial
outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
- Statistics: Num rows: 6144 Data size: 173221 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 6144 Data size: 173216 Basic stats: COMPLETE Column stats: NONE
Filter Operator
Filter Vectorization:
className: VectorFilterOperator
native: true
predicateExpression: FilterLongColGreaterLongScalar(col 9:bigint, val 1)
predicate: (_col9 > 1L) (type: boolean)
- Statistics: Num rows: 2048 Data size: 57740 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 2048 Data size: 57738 Basic stats: COMPLETE Column stats: NONE
Select Operator
expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: decimal(11,5)), _col3 (type: decimal(11,5)), _col4 (type: decimal(21,5)), _col5 (type: bigint), _col6 (type: decimal(16,0)), _col7 (type: decimal(16,0)), _col8 (type: decimal(26,0))
outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -555,13 +555,13 @@ STAGE PLANS:
className: VectorSelectOperator
native: true
projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6, 7, 8]
- Statistics: Num rows: 2048 Data size: 57740 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 2048 Data size: 57738 Basic stats: COMPLETE Column stats: NONE
File Output Operator
compressed: false
File Sink Vectorization:
className: VectorFileSinkOperator
native: false
- Statistics: Num rows: 2048 Data size: 57740 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 2048 Data size: 57738 Basic stats: COMPLETE Column stats: NONE
table:
input format: org.apache.hadoop.mapred.SequenceFileInputFormat
output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -653,7 +653,7 @@ STAGE PLANS:
Map Operator Tree:
TableScan
alias: decimal_vgby_small
- Statistics: Num rows: 12289 Data size: 346472 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 12289 Data size: 346462 Basic stats: COMPLETE Column stats: NONE
TableScan Vectorization:
native: true
vectorizationSchemaColumns: [0:cdouble:double, 1:cdecimal1:decimal(11,5)/DECIMAL_64, 2:cdecimal2:decimal(16,0)/DECIMAL_64, 3:cint:int, 4:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
@@ -665,7 +665,7 @@ STAGE PLANS:
native: true
projectedOutputColumnNums: [3, 1, 2, 6, 9, 7, 12]
selectExpressions: CastDecimalToDouble(col 5:decimal(11,5))(children: ConvertDecimal64ToDecimal(col 1:decimal(11,5)/DECIMAL_64) -> 5:decimal(11,5)) -> 6:double, DoubleColMultiplyDoubleColumn(col 7:double, col 8:double)(children: CastDecimalToDouble(col 5:decimal(11,5))(children: ConvertDecimal64ToDecimal(col 1:decimal(11,5)/DECIMAL_64) -> 5:decimal(11,5)) -> 7:double, CastDecimalToDouble(col 5:decimal(11,5))(children: ConvertDecimal64ToDecimal(col 1:decimal(11,5)/DECIMAL_64) -> 5:decimal(11,5)) -> 8:double) -> 9:double, CastDecimalToDouble(col 10:decimal(16,0))(children: ConvertDecimal64ToDecimal(col 2:decimal(16,0)/DECIMAL_64) -> 10:decimal(16,0)) -> 7:double, DoubleColMultiplyDoubleColumn(col 8:double, col 11:double)(children: CastDecimalToDouble(col 10:decimal(16,0))(children: ConvertDecimal64ToDecimal(col 2:decimal(16,0)/DECIMAL_64) -> 10:decimal(16,0)) -> 8:double, CastDecimalToDouble(col 10:decimal(16,0))(children: ConvertDecimal64ToDecimal(col 2:decima
l(16,0)/DECIMAL_64) -> 10:decimal(16,0)) -> 11:double) -> 12:double
- Statistics: Num rows: 12289 Data size: 346472 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 12289 Data size: 346462 Basic stats: COMPLETE Column stats: NONE
Group By Operator
aggregations: count(_col1), max(_col1), min(_col1), sum(_col1), sum(_col4), sum(_col3), count(_col2), max(_col2), min(_col2), sum(_col2), sum(_col6), sum(_col5), count()
Group By Vectorization:
@@ -679,7 +679,7 @@ STAGE PLANS:
keys: _col0 (type: int)
mode: hash
outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13
- Statistics: Num rows: 12289 Data size: 346472 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 12289 Data size: 346462 Basic stats: COMPLETE Column stats: NONE
Reduce Output Operator
key expressions: _col0 (type: int)
sort order: +
@@ -690,7 +690,7 @@ STAGE PLANS:
native: true
nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
valueColumnNums: [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13]
- Statistics: Num rows: 12289 Data size: 346472 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 12289 Data size: 346462 Basic stats: COMPLETE Column stats: NONE
value expressions: _col1 (type: bigint), _col2 (type: decimal(11,5)), _col3 (type: decimal(11,5)), _col4 (type: decimal(21,5)), _col5 (type: double), _col6 (type: double), _col7 (type: bigint), _col8 (type: decimal(16,0)), _col9 (type: decimal(16,0)), _col10 (type: decimal(26,0)), _col11 (type: double), _col12 (type: double), _col13 (type: bigint)
Execution mode: vectorized
Map Vectorization:
@@ -737,14 +737,14 @@ STAGE PLANS:
keys: KEY._col0 (type: int)
mode: mergepartial
outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13
- Statistics: Num rows: 6144 Data size: 173221 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 6144 Data size: 173216 Basic stats: COMPLETE Column stats: NONE
Filter Operator
Filter Vectorization:
className: VectorFilterOperator
native: true
predicateExpression: FilterLongColGreaterLongScalar(col 13:bigint, val 1)
predicate: (_col13 > 1L) (type: boolean)
- Statistics: Num rows: 2048 Data size: 57740 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 2048 Data size: 57738 Basic stats: COMPLETE Column stats: NONE
Select Operator
expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: decimal(11,5)), _col3 (type: decimal(11,5)), _col4 (type: decimal(21,5)), (CAST( _col4 AS decimal(15,9)) / _col1) (type: decimal(35,29)), power(((_col5 - ((_col6 * _col6) / _col1)) / _col1), 0.5) (type: double), power(((_col5 - ((_col6 * _col6) / _col1)) / CASE WHEN ((_col1 = 1L)) THEN (null) ELSE ((_col1 - 1)) END), 0.5) (type: double), _col7 (type: bigint), _col8 (type: decimal(16,0)), _col9 (type: decimal(16,0)), _col10 (type: decimal(26,0)), CAST( (CAST( _col10 AS decimal(20,4)) / _col7) AS decimal(20,4)) (type: decimal(20,4)), power(((_col11 - ((_col12 * _col12) / _col7)) / _col7), 0.5) (type: double), power(((_col11 - ((_col12 * _col12) / _col7)) / CASE WHEN ((_col7 = 1L)) THEN (null) ELSE ((_col7 - 1)) END), 0.5) (type: double)
outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14
@@ -753,13 +753,13 @@ STAGE PLANS:
native: true
projectedOutputColumnNums: [0, 1, 2, 3, 4, 16, 17, 18, 7, 8, 9, 10, 23, 19, 25]
selectExpressions: DecimalColDivideDecimalColumn(col 14:decimal(15,9), col 15:decimal(19,0))(children: CastDecimalToDecimal(col 4:decimal(21,5)) -> 14:decimal(15,9), CastLongToDecimal(col 1:bigint) -> 15:decimal(19,0)) -> 16:decimal(35,29), FuncPowerDoubleToDouble(col 18:double)(children: DoubleColDivideLongColumn(col 17:double, col 1:bigint)(children: DoubleColSubtractDoubleColumn(col 5:double, col 18:double)(children: DoubleColDivideLongColumn(col 17:double, col 1:bigint)(children: DoubleColMultiplyDoubleColumn(col 6:double, col 6:double) -> 17:double) -> 18:double) -> 17:double) -> 18:double) -> 17:double, FuncPowerDoubleToDouble(col 19:double)(children: DoubleColDivideLongColumn(col 18:double, col 22:bigint)(children: DoubleColSubtractDoubleColumn(col 5:double, col 19:double)(children: DoubleColDivideLongColumn(col 18:double, col 1:bigint)(children: DoubleColMultiplyDoubleColumn(col 6:double, col 6:double) -> 18:double) -> 19:double) -> 18:double, IfExprN
ullCondExpr(col 20:boolean, null, col 21:bigint)(children: LongColEqualLongScalar(col 1:bigint, val 1) -> 20:boolean, LongColSubtractLongScalar(col 1:bigint, val 1) -> 21:bigint) -> 22:bigint) -> 19:double) -> 18:double, CastDecimalToDecimal(col 24:decimal(38,22))(children: DecimalColDivideDecimalColumn(col 23:decimal(20,4), col 15:decimal(19,0))(children: CastDecimalToDecimal(col 10:decimal(26,0)) -> 23:decimal(20,4), CastLongToDecimal(col 7:bigint) -> 15:decimal(19,0)) -> 24:decimal(38,22)) -> 23:decimal(20,4), FuncPowerDoubleToDouble(col 25:double)(children: DoubleColDivideLongColumn(col 19:double, col 7:bigint)(children: DoubleColSubtractDoubleColumn(col 11:double, col 25:double)(children: DoubleColDivideLongColumn(col 19:double, col 7:bigint)(children: DoubleColMultiplyDoubleColumn(col 12:double, col 12:double) -> 19:double) -> 25:double) -> 19:double) -> 25:double) -> 19:double, FuncPowerDoubleToDouble(col 26:double)(children: DoubleColDivideLongColumn(col 25:double, col 28:bi
gint)(children: DoubleColSubtractDoubleColumn(col 11:double, col 26:double)(children: DoubleColDivideLongColumn(col 25:double, col 7:bigint)(children: DoubleColMultiplyDoubleColumn(col 12:double, col 12:double) -> 25:double) -> 26:double) -> 25:double, IfExprNullCondExpr(col 22:boolean, null, col 27:bigint)(children: LongColEqualLongScalar(col 7:bigint, val 1) -> 22:boolean, LongColSubtractLongScalar(col 7:bigint, val 1) -> 27:bigint) -> 28:bigint) -> 26:double) -> 25:double
- Statistics: Num rows: 2048 Data size: 57740 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 2048 Data size: 57738 Basic stats: COMPLETE Column stats: NONE
File Output Operator
compressed: false
File Sink Vectorization:
className: VectorFileSinkOperator
native: false
- Statistics: Num rows: 2048 Data size: 57740 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 2048 Data size: 57738 Basic stats: COMPLETE Column stats: NONE
table:
input format: org.apache.hadoop.mapred.SequenceFileInputFormat
output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -796,7 +796,7 @@ POSTHOOK: Input: default@decimal_vgby_small
626923679 1024 9723.40270 -9778.95135 10541.05247 10.29399655273437500000000000000 5742.091453325365 5744.897264122335 1024 11646 -11712 12641 12.3447 6877.306686989158 6880.6672084147185
6981 2 -515.62107 -515.62107 -1031.24214 -515.62107000000000000000000000000 0.0 0.0 3 6984454 -618 6983218 2327739.3333 3292794.518850853 4032833.1995089175
762 1 1531.21941 1531.21941 1531.21941 1531.21941000000000000000000000000 0.0 NULL 2 6984454 1834 6986288 3493144.0000 3491310.0 4937457.95244881
-NULL 3072 9318.43514 -4298.15135 5018444.11392 1633.60811000000000000000000000000 5695.4830839098695 5696.410309489299 3072 11161 -5148 6010880 1956.6667 6821.647911041892 6822.758476439734
+NULL 3072 9318.43514 -4298.15135 5018444.11392 NULL 5695.4830839098695 5696.410309489299 3072 11161 -5148 6010880 1956.6667 6821.647911041892 6822.758476439734
PREHOOK: query: SELECT SUM(HASH(*))
FROM (SELECT cint,
COUNT(cdecimal1), MAX(cdecimal1), MIN(cdecimal1), SUM(cdecimal1), AVG(cdecimal1), STDDEV_POP(cdecimal1), STDDEV_SAMP(cdecimal1),
@@ -815,4 +815,4 @@ FROM (SELECT cint,
POSTHOOK: type: QUERY
POSTHOOK: Input: default@decimal_vgby_small
#### A masked pattern was here ####
-96966670826
+96673467876
http://git-wip-us.apache.org/repos/asf/hive/blob/25aaf7db/ql/src/test/results/clientpositive/timestamp_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/timestamp_1.q.out b/ql/src/test/results/clientpositive/timestamp_1.q.out
index d3ca5cf..fab69ec 100644
--- a/ql/src/test/results/clientpositive/timestamp_1.q.out
+++ b/ql/src/test/results/clientpositive/timestamp_1.q.out
@@ -46,7 +46,7 @@ POSTHOOK: query: select cast(t as tinyint) from timestamp_1 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_1
#### A masked pattern was here ####
-77
+NULL
PREHOOK: query: select cast(t as smallint) from timestamp_1 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_1
@@ -55,7 +55,7 @@ POSTHOOK: query: select cast(t as smallint) from timestamp_1 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_1
#### A masked pattern was here ####
--4787
+NULL
PREHOOK: query: select cast(t as int) from timestamp_1 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_1
@@ -129,7 +129,7 @@ POSTHOOK: query: select cast(t as tinyint) from timestamp_1 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_1
#### A masked pattern was here ####
-77
+NULL
PREHOOK: query: select cast(t as smallint) from timestamp_1 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_1
@@ -138,7 +138,7 @@ POSTHOOK: query: select cast(t as smallint) from timestamp_1 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_1
#### A masked pattern was here ####
--4787
+NULL
PREHOOK: query: select cast(t as int) from timestamp_1 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_1
@@ -212,7 +212,7 @@ POSTHOOK: query: select cast(t as tinyint) from timestamp_1 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_1
#### A masked pattern was here ####
-77
+NULL
PREHOOK: query: select cast(t as smallint) from timestamp_1 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_1
@@ -221,7 +221,7 @@ POSTHOOK: query: select cast(t as smallint) from timestamp_1 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_1
#### A masked pattern was here ####
--4787
+NULL
PREHOOK: query: select cast(t as int) from timestamp_1 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_1
@@ -295,7 +295,7 @@ POSTHOOK: query: select cast(t as tinyint) from timestamp_1 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_1
#### A masked pattern was here ####
-77
+NULL
PREHOOK: query: select cast(t as smallint) from timestamp_1 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_1
@@ -304,7 +304,7 @@ POSTHOOK: query: select cast(t as smallint) from timestamp_1 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_1
#### A masked pattern was here ####
--4787
+NULL
PREHOOK: query: select cast(t as int) from timestamp_1 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_1
@@ -378,7 +378,7 @@ POSTHOOK: query: select cast(t as tinyint) from timestamp_1 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_1
#### A masked pattern was here ####
-77
+NULL
PREHOOK: query: select cast(t as smallint) from timestamp_1 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_1
@@ -387,7 +387,7 @@ POSTHOOK: query: select cast(t as smallint) from timestamp_1 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_1
#### A masked pattern was here ####
--4787
+NULL
PREHOOK: query: select cast(t as int) from timestamp_1 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_1
@@ -461,7 +461,7 @@ POSTHOOK: query: select cast(t as tinyint) from timestamp_1 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_1
#### A masked pattern was here ####
-77
+NULL
PREHOOK: query: select cast(t as smallint) from timestamp_1 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_1
@@ -470,7 +470,7 @@ POSTHOOK: query: select cast(t as smallint) from timestamp_1 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_1
#### A masked pattern was here ####
--4787
+NULL
PREHOOK: query: select cast(t as int) from timestamp_1 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_1
http://git-wip-us.apache.org/repos/asf/hive/blob/25aaf7db/ql/src/test/results/clientpositive/timestamp_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/timestamp_2.q.out b/ql/src/test/results/clientpositive/timestamp_2.q.out
index f9bfb09..9a05dfe 100644
--- a/ql/src/test/results/clientpositive/timestamp_2.q.out
+++ b/ql/src/test/results/clientpositive/timestamp_2.q.out
@@ -46,7 +46,7 @@ POSTHOOK: query: select cast(t as tinyint) from timestamp_2 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_2
#### A masked pattern was here ####
-77
+NULL
PREHOOK: query: select cast(t as smallint) from timestamp_2 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_2
@@ -55,7 +55,7 @@ POSTHOOK: query: select cast(t as smallint) from timestamp_2 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_2
#### A masked pattern was here ####
--4787
+NULL
PREHOOK: query: select cast(t as int) from timestamp_2 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_2
@@ -129,7 +129,7 @@ POSTHOOK: query: select cast(t as tinyint) from timestamp_2 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_2
#### A masked pattern was here ####
-77
+NULL
PREHOOK: query: select cast(t as smallint) from timestamp_2 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_2
@@ -138,7 +138,7 @@ POSTHOOK: query: select cast(t as smallint) from timestamp_2 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_2
#### A masked pattern was here ####
--4787
+NULL
PREHOOK: query: select cast(t as int) from timestamp_2 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_2
@@ -212,7 +212,7 @@ POSTHOOK: query: select cast(t as tinyint) from timestamp_2 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_2
#### A masked pattern was here ####
-77
+NULL
PREHOOK: query: select cast(t as smallint) from timestamp_2 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_2
@@ -221,7 +221,7 @@ POSTHOOK: query: select cast(t as smallint) from timestamp_2 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_2
#### A masked pattern was here ####
--4787
+NULL
PREHOOK: query: select cast(t as int) from timestamp_2 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_2
@@ -295,7 +295,7 @@ POSTHOOK: query: select cast(t as tinyint) from timestamp_2 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_2
#### A masked pattern was here ####
-77
+NULL
PREHOOK: query: select cast(t as smallint) from timestamp_2 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_2
@@ -304,7 +304,7 @@ POSTHOOK: query: select cast(t as smallint) from timestamp_2 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_2
#### A masked pattern was here ####
--4787
+NULL
PREHOOK: query: select cast(t as int) from timestamp_2 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_2
@@ -378,7 +378,7 @@ POSTHOOK: query: select cast(t as tinyint) from timestamp_2 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_2
#### A masked pattern was here ####
-77
+NULL
PREHOOK: query: select cast(t as smallint) from timestamp_2 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_2
@@ -387,7 +387,7 @@ POSTHOOK: query: select cast(t as smallint) from timestamp_2 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_2
#### A masked pattern was here ####
--4787
+NULL
PREHOOK: query: select cast(t as int) from timestamp_2 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_2
@@ -461,7 +461,7 @@ POSTHOOK: query: select cast(t as tinyint) from timestamp_2 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_2
#### A masked pattern was here ####
-77
+NULL
PREHOOK: query: select cast(t as smallint) from timestamp_2 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_2
@@ -470,7 +470,7 @@ POSTHOOK: query: select cast(t as smallint) from timestamp_2 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_2
#### A masked pattern was here ####
--4787
+NULL
PREHOOK: query: select cast(t as int) from timestamp_2 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_2
http://git-wip-us.apache.org/repos/asf/hive/blob/25aaf7db/ql/src/test/results/clientpositive/timestamp_3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/timestamp_3.q.out b/ql/src/test/results/clientpositive/timestamp_3.q.out
index 0664abf..6d59269 100644
--- a/ql/src/test/results/clientpositive/timestamp_3.q.out
+++ b/ql/src/test/results/clientpositive/timestamp_3.q.out
@@ -46,7 +46,7 @@ POSTHOOK: query: select cast(t as tinyint) from timestamp_3 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_3
#### A masked pattern was here ####
-48
+NULL
PREHOOK: query: select cast(t as smallint) from timestamp_3 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_3
@@ -55,7 +55,7 @@ POSTHOOK: query: select cast(t as smallint) from timestamp_3 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_3
#### A masked pattern was here ####
--31184
+NULL
PREHOOK: query: select cast(t as int) from timestamp_3 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_3
http://git-wip-us.apache.org/repos/asf/hive/blob/25aaf7db/ql/src/test/results/clientpositive/vector_decimal_aggregate.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_decimal_aggregate.q.out b/ql/src/test/results/clientpositive/vector_decimal_aggregate.q.out
index 16c80f0..04c534e 100644
--- a/ql/src/test/results/clientpositive/vector_decimal_aggregate.q.out
+++ b/ql/src/test/results/clientpositive/vector_decimal_aggregate.q.out
@@ -379,7 +379,7 @@ STAGE PLANS:
Map Operator Tree:
TableScan
alias: decimal_vgby_small
- Statistics: Num rows: 12289 Data size: 346472 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 12289 Data size: 346462 Basic stats: COMPLETE Column stats: NONE
TableScan Vectorization:
native: true
vectorizationSchemaColumns: [0:cdouble:double, 1:cdecimal1:decimal(11,5)/DECIMAL_64, 2:cdecimal2:decimal(16,0)/DECIMAL_64, 3:cint:int, 4:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
@@ -390,7 +390,7 @@ STAGE PLANS:
className: VectorSelectOperator
native: true
projectedOutputColumnNums: [1, 2, 3]
- Statistics: Num rows: 12289 Data size: 346472 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 12289 Data size: 346462 Basic stats: COMPLETE Column stats: NONE
Group By Operator
aggregations: count(cdecimal1), max(cdecimal1), min(cdecimal1), sum(cdecimal1), count(cdecimal2), max(cdecimal2), min(cdecimal2), sum(cdecimal2), count()
Group By Vectorization:
@@ -404,7 +404,7 @@ STAGE PLANS:
keys: cint (type: int)
mode: hash
outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
- Statistics: Num rows: 12289 Data size: 346472 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 12289 Data size: 346462 Basic stats: COMPLETE Column stats: NONE
Reduce Output Operator
key expressions: _col0 (type: int)
sort order: +
@@ -414,7 +414,7 @@ STAGE PLANS:
native: false
nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false
- Statistics: Num rows: 12289 Data size: 346472 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 12289 Data size: 346462 Basic stats: COMPLETE Column stats: NONE
value expressions: _col1 (type: bigint), _col2 (type: decimal(11,5)), _col3 (type: decimal(11,5)), _col4 (type: decimal(21,5)), _col5 (type: bigint), _col6 (type: decimal(16,0)), _col7 (type: decimal(16,0)), _col8 (type: decimal(26,0)), _col9 (type: bigint)
Execution mode: vectorized
Map Vectorization:
@@ -442,17 +442,17 @@ STAGE PLANS:
keys: KEY._col0 (type: int)
mode: mergepartial
outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
- Statistics: Num rows: 6144 Data size: 173221 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 6144 Data size: 173216 Basic stats: COMPLETE Column stats: NONE
Filter Operator
predicate: (_col9 > 1L) (type: boolean)
- Statistics: Num rows: 2048 Data size: 57740 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 2048 Data size: 57738 Basic stats: COMPLETE Column stats: NONE
Select Operator
expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: decimal(11,5)), _col3 (type: decimal(11,5)), _col4 (type: decimal(21,5)), _col5 (type: bigint), _col6 (type: decimal(16,0)), _col7 (type: decimal(16,0)), _col8 (type: decimal(26,0))
outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
- Statistics: Num rows: 2048 Data size: 57740 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 2048 Data size: 57738 Basic stats: COMPLETE Column stats: NONE
File Output Operator
compressed: false
- Statistics: Num rows: 2048 Data size: 57740 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 2048 Data size: 57738 Basic stats: COMPLETE Column stats: NONE
table:
input format: org.apache.hadoop.mapred.SequenceFileInputFormat
output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -539,7 +539,7 @@ STAGE PLANS:
Map Operator Tree:
TableScan
alias: decimal_vgby_small
- Statistics: Num rows: 12289 Data size: 346472 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 12289 Data size: 346462 Basic stats: COMPLETE Column stats: NONE
TableScan Vectorization:
native: true
vectorizationSchemaColumns: [0:cdouble:double, 1:cdecimal1:decimal(11,5)/DECIMAL_64, 2:cdecimal2:decimal(16,0)/DECIMAL_64, 3:cint:int, 4:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
@@ -551,7 +551,7 @@ STAGE PLANS:
native: true
projectedOutputColumnNums: [3, 1, 2, 6, 9, 7, 12]
selectExpressions: CastDecimalToDouble(col 5:decimal(11,5))(children: ConvertDecimal64ToDecimal(col 1:decimal(11,5)/DECIMAL_64) -> 5:decimal(11,5)) -> 6:double, DoubleColMultiplyDoubleColumn(col 7:double, col 8:double)(children: CastDecimalToDouble(col 5:decimal(11,5))(children: ConvertDecimal64ToDecimal(col 1:decimal(11,5)/DECIMAL_64) -> 5:decimal(11,5)) -> 7:double, CastDecimalToDouble(col 5:decimal(11,5))(children: ConvertDecimal64ToDecimal(col 1:decimal(11,5)/DECIMAL_64) -> 5:decimal(11,5)) -> 8:double) -> 9:double, CastDecimalToDouble(col 10:decimal(16,0))(children: ConvertDecimal64ToDecimal(col 2:decimal(16,0)/DECIMAL_64) -> 10:decimal(16,0)) -> 7:double, DoubleColMultiplyDoubleColumn(col 8:double, col 11:double)(children: CastDecimalToDouble(col 10:decimal(16,0))(children: ConvertDecimal64ToDecimal(col 2:decimal(16,0)/DECIMAL_64) -> 10:decimal(16,0)) -> 8:double, CastDecimalToDouble(col 10:decimal(16,0))(children: ConvertDecimal64ToDecimal(col 2:decimal(16,0
)/DECIMAL_64) -> 10:decimal(16,0)) -> 11:double) -> 12:double
- Statistics: Num rows: 12289 Data size: 346472 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 12289 Data size: 346462 Basic stats: COMPLETE Column stats: NONE
Group By Operator
aggregations: count(_col1), max(_col1), min(_col1), sum(_col1), sum(_col4), sum(_col3), count(_col2), max(_col2), min(_col2), sum(_col2), sum(_col6), sum(_col5), count()
Group By Vectorization:
@@ -565,7 +565,7 @@ STAGE PLANS:
keys: _col0 (type: int)
mode: hash
outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13
- Statistics: Num rows: 12289 Data size: 346472 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 12289 Data size: 346462 Basic stats: COMPLETE Column stats: NONE
Reduce Output Operator
key expressions: _col0 (type: int)
sort order: +
@@ -575,7 +575,7 @@ STAGE PLANS:
native: false
nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false
- Statistics: Num rows: 12289 Data size: 346472 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 12289 Data size: 346462 Basic stats: COMPLETE Column stats: NONE
value expressions: _col1 (type: bigint), _col2 (type: decimal(11,5)), _col3 (type: decimal(11,5)), _col4 (type: decimal(21,5)), _col5 (type: double), _col6 (type: double), _col7 (type: bigint), _col8 (type: decimal(16,0)), _col9 (type: decimal(16,0)), _col10 (type: decimal(26,0)), _col11 (type: double), _col12 (type: double), _col13 (type: bigint)
Execution mode: vectorized
Map Vectorization:
@@ -603,17 +603,17 @@ STAGE PLANS:
keys: KEY._col0 (type: int)
mode: mergepartial
outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13
- Statistics: Num rows: 6144 Data size: 173221 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 6144 Data size: 173216 Basic stats: COMPLETE Column stats: NONE
Filter Operator
predicate: (_col13 > 1L) (type: boolean)
- Statistics: Num rows: 2048 Data size: 57740 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 2048 Data size: 57738 Basic stats: COMPLETE Column stats: NONE
Select Operator
expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: decimal(11,5)), _col3 (type: decimal(11,5)), _col4 (type: decimal(21,5)), (CAST( _col4 AS decimal(15,9)) / _col1) (type: decimal(35,29)), power(((_col5 - ((_col6 * _col6) / _col1)) / _col1), 0.5) (type: double), power(((_col5 - ((_col6 * _col6) / _col1)) / CASE WHEN ((_col1 = 1L)) THEN (null) ELSE ((_col1 - 1)) END), 0.5) (type: double), _col7 (type: bigint), _col8 (type: decimal(16,0)), _col9 (type: decimal(16,0)), _col10 (type: decimal(26,0)), CAST( (CAST( _col10 AS decimal(20,4)) / _col7) AS decimal(20,4)) (type: decimal(20,4)), power(((_col11 - ((_col12 * _col12) / _col7)) / _col7), 0.5) (type: double), power(((_col11 - ((_col12 * _col12) / _col7)) / CASE WHEN ((_col7 = 1L)) THEN (null) ELSE ((_col7 - 1)) END), 0.5) (type: double)
outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14
- Statistics: Num rows: 2048 Data size: 57740 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 2048 Data size: 57738 Basic stats: COMPLETE Column stats: NONE
File Output Operator
compressed: false
- Statistics: Num rows: 2048 Data size: 57740 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 2048 Data size: 57738 Basic stats: COMPLETE Column stats: NONE
table:
input format: org.apache.hadoop.mapred.SequenceFileInputFormat
output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
http://git-wip-us.apache.org/repos/asf/hive/blob/25aaf7db/vector-code-gen/src/org/apache/hadoop/hive/tools/GenVectorCode.java
----------------------------------------------------------------------
diff --git a/vector-code-gen/src/org/apache/hadoop/hive/tools/GenVectorCode.java b/vector-code-gen/src/org/apache/hadoop/hive/tools/GenVectorCode.java
index b5220a0..45fa739 100644
--- a/vector-code-gen/src/org/apache/hadoop/hive/tools/GenVectorCode.java
+++ b/vector-code-gen/src/org/apache/hadoop/hive/tools/GenVectorCode.java
@@ -1035,8 +1035,6 @@ public class GenVectorCode extends Task {
"", "", ""},
{"ColumnUnaryFunc", "CastLongToBooleanVia", "long", "long", "MathExpr.toBool", "",
"", "", ""},
- {"ColumnUnaryFunc", "CastDateToBooleanVia", "long", "long", "MathExpr.toBool", "",
- "", "", "date"},
// Boolean to long is done with an IdentityExpression
// Boolean to double is done with standard Long to Double cast
[2/2] hive git commit: HIVE-19498: Vectorization: CAST expressions
produce wrong results (Matt McCline, reviewed by Teddy Choi)
Posted by mm...@apache.org.
HIVE-19498: Vectorization: CAST expressions produce wrong results (Matt McCline, reviewed by Teddy Choi)
Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/25aaf7db
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/25aaf7db
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/25aaf7db
Branch: refs/heads/master
Commit: 25aaf7db0d62d6007c79213a33dae0fb8ac9a7be
Parents: 2c848ef
Author: Matt McCline <mm...@hortonworks.com>
Authored: Thu May 24 16:41:42 2018 -0500
Committer: Matt McCline <mm...@hortonworks.com>
Committed: Thu May 24 16:41:42 2018 -0500
----------------------------------------------------------------------
.../org/apache/hadoop/hive/conf/HiveConf.java | 4 +
.../UDAFTemplates/VectorUDAFAvgDecimal.txt | 2 +-
.../UDAFTemplates/VectorUDAFAvgDecimalMerge.txt | 2 +-
.../UDAFTemplates/VectorUDAFMinMaxDecimal.txt | 2 +-
.../exec/vector/VectorExpressionDescriptor.java | 72 +--
.../exec/vector/VectorHashKeyWrapperBatch.java | 2 +-
.../ql/exec/vector/VectorizationContext.java | 26 +-
.../ql/exec/vector/VectorizedBatchUtil.java | 4 +
.../vector/expressions/CastDateToBoolean.java | 61 +++
.../expressions/CastDecimalToDecimal.java | 2 +-
.../vector/expressions/CastDoubleToDecimal.java | 15 +-
.../vector/expressions/CastFloatToDecimal.java | 65 +++
.../vector/expressions/CastLongToDecimal.java | 2 +-
.../vector/expressions/CastStringToDecimal.java | 2 +-
.../vector/expressions/CastTimestampToLong.java | 60 ++-
.../expressions/NullVectorExpression.java | 56 +++
.../aggregates/VectorUDAFSumDecimal.java | 2 +-
.../VectorUDAFSumDecimal64ToDecimal.java | 2 +-
.../VectorPTFEvaluatorDecimalFirstValue.java | 2 +-
.../exec/vector/ptf/VectorPTFGroupBatches.java | 2 +-
.../hive/ql/plan/ExprNodeGenericFuncDesc.java | 5 +-
.../apache/hadoop/hive/ql/udf/UDFToBoolean.java | 4 +-
.../apache/hadoop/hive/ql/udf/UDFToByte.java | 7 +-
.../apache/hadoop/hive/ql/udf/UDFToInteger.java | 7 +-
.../apache/hadoop/hive/ql/udf/UDFToShort.java | 7 +-
.../ql/exec/vector/VectorRandomRowSource.java | 68 ++-
.../expressions/TestVectorCastStatement.java | 502 +++++++++++++++++++
.../vector/expressions/TestVectorTypeCasts.java | 4 +
.../llap/vector_decimal_aggregate.q.out | 4 +-
.../clientpositive/spark/timestamp_1.q.out | 24 +-
.../clientpositive/spark/timestamp_2.q.out | 24 +-
.../clientpositive/spark/timestamp_3.q.out | 4 +-
.../spark/vector_decimal_aggregate.q.out | 36 +-
.../results/clientpositive/timestamp_1.q.out | 24 +-
.../results/clientpositive/timestamp_2.q.out | 24 +-
.../results/clientpositive/timestamp_3.q.out | 4 +-
.../vector_decimal_aggregate.q.out | 32 +-
.../apache/hadoop/hive/tools/GenVectorCode.java | 2 -
38 files changed, 970 insertions(+), 197 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hive/blob/25aaf7db/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index 931533a..7ed3a9c 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -3554,6 +3554,10 @@ public class HiveConf extends Configuration {
"1. chosen : use VectorUDFAdaptor for a small set of UDFs that were chosen for good performance\n" +
"2. all : use VectorUDFAdaptor for all UDFs"
),
+ HIVE_TEST_VECTOR_ADAPTOR_OVERRIDE("hive.test.vectorized.adaptor.override", false,
+ "internal use only, used to force always using the VectorUDFAdaptor.\n" +
+ "The default is false, of course",
+ true),
HIVE_VECTORIZATION_PTF_ENABLED("hive.vectorized.execution.ptf.enabled", true,
"This flag should be set to true to enable vectorized mode of the PTF of query execution.\n" +
"The default value is true."),
http://git-wip-us.apache.org/repos/asf/hive/blob/25aaf7db/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFAvgDecimal.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFAvgDecimal.txt b/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFAvgDecimal.txt
index fa72171..f512639 100644
--- a/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFAvgDecimal.txt
+++ b/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFAvgDecimal.txt
@@ -522,7 +522,7 @@ public class <ClassName> extends VectorAggregateExpression {
fields[AVERAGE_COUNT_FIELD_INDEX].isNull[batchIndex] = false;
((LongColumnVector) fields[AVERAGE_COUNT_FIELD_INDEX]).vector[batchIndex] = myagg.count;
fields[AVERAGE_SUM_FIELD_INDEX].isNull[batchIndex] = false;
- ((DecimalColumnVector) fields[AVERAGE_SUM_FIELD_INDEX]).vector[batchIndex].set(myagg.sum);
+ ((DecimalColumnVector) fields[AVERAGE_SUM_FIELD_INDEX]).set(batchIndex, myagg.sum);
// NULL out useless source field.
ColumnVector sourceColVector = (ColumnVector) fields[AVERAGE_SOURCE_FIELD_INDEX];
http://git-wip-us.apache.org/repos/asf/hive/blob/25aaf7db/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFAvgDecimalMerge.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFAvgDecimalMerge.txt b/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFAvgDecimalMerge.txt
index e273d07..5fe9256 100644
--- a/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFAvgDecimalMerge.txt
+++ b/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFAvgDecimalMerge.txt
@@ -532,7 +532,7 @@ public class <ClassName> extends VectorAggregateExpression {
fields[AVERAGE_COUNT_FIELD_INDEX].isNull[batchIndex] = false;
((LongColumnVector) fields[AVERAGE_COUNT_FIELD_INDEX]).vector[batchIndex] = myagg.mergeCount;
fields[AVERAGE_SUM_FIELD_INDEX].isNull[batchIndex] = false;
- ((DecimalColumnVector) fields[AVERAGE_SUM_FIELD_INDEX]).vector[batchIndex].set(myagg.mergeSum);
+ ((DecimalColumnVector) fields[AVERAGE_SUM_FIELD_INDEX]).set(batchIndex, myagg.mergeSum);
// NULL out useless source field.
ColumnVector sourceColVector = (ColumnVector) fields[AVERAGE_SOURCE_FIELD_INDEX];
http://git-wip-us.apache.org/repos/asf/hive/blob/25aaf7db/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxDecimal.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxDecimal.txt b/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxDecimal.txt
index 9fe85d3..9c8ebcc 100644
--- a/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxDecimal.txt
+++ b/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxDecimal.txt
@@ -471,6 +471,6 @@ public class <ClassName> extends VectorAggregateExpression {
return;
}
outputColVector.isNull[batchIndex] = false;
- outputColVector.vector[batchIndex].set(myagg.value);
+ outputColVector.set(batchIndex, myagg.value);
}
}
http://git-wip-us.apache.org/repos/asf/hive/blob/25aaf7db/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorExpressionDescriptor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorExpressionDescriptor.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorExpressionDescriptor.java
index 2d73ab4..2f8a419 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorExpressionDescriptor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorExpressionDescriptor.java
@@ -63,39 +63,41 @@ public class VectorExpressionDescriptor {
// INTERVAL_DAY_TIME
//
public enum ArgumentType {
- NONE (0x000),
- INT_FAMILY (0x001),
- FLOAT_FAMILY (0x002),
- DECIMAL (0x004),
- STRING (0x008),
- CHAR (0x010),
- VARCHAR (0x020),
+ NONE (0x000000L),
+ INT_FAMILY (0x000001L),
+ FLOAT (0x000002L),
+ DOUBLE (0x000004L),
+ FLOAT_FAMILY (FLOAT.value | DOUBLE.value),
+ DECIMAL (0x000008L),
+ STRING (0x000010L),
+ CHAR (0x000020L),
+ VARCHAR (0x000040L),
STRING_FAMILY (STRING.value | CHAR.value | VARCHAR.value),
- DATE (0x040),
- TIMESTAMP (0x080),
- INTERVAL_YEAR_MONTH (0x100),
- INTERVAL_DAY_TIME (0x200),
- BINARY (0x400),
- STRUCT (0x800),
- DECIMAL_64 (0x1000),
- LIST (0x2000),
- MAP (0x4000),
- VOID (0x8000),
+ DATE (0x000080L),
+ TIMESTAMP (0x000100L),
+ INTERVAL_YEAR_MONTH (0x000200L),
+ INTERVAL_DAY_TIME (0x000400L),
+ BINARY (0x000800L),
+ STRUCT (0x001000L),
+ DECIMAL_64 (0x002000L),
+ LIST (0x004000L),
+ MAP (0x008000L),
+ VOID (0x010000L),
INT_DECIMAL_64_FAMILY (INT_FAMILY.value | DECIMAL_64.value),
DATETIME_FAMILY (DATE.value | TIMESTAMP.value),
INTERVAL_FAMILY (INTERVAL_YEAR_MONTH.value | INTERVAL_DAY_TIME.value),
INT_INTERVAL_YEAR_MONTH (INT_FAMILY.value | INTERVAL_YEAR_MONTH.value),
INT_DATE_INTERVAL_YEAR_MONTH (INT_FAMILY.value | DATE.value | INTERVAL_YEAR_MONTH.value),
STRING_DATETIME_FAMILY (STRING_FAMILY.value | DATETIME_FAMILY.value),
- ALL_FAMILY (0xFFFF);
+ ALL_FAMILY (0xFFFFFFL);
- private final int value;
+ private final long value;
- ArgumentType(int val) {
+ ArgumentType(long val) {
this.value = val;
}
- public int getValue() {
+ public long getValue() {
return value;
}
@@ -161,34 +163,6 @@ public class VectorExpressionDescriptor {
public boolean isSameTypeOrFamily(ArgumentType other) {
return ((value & other.value) != 0);
}
-
- public static String getVectorColumnSimpleName(ArgumentType argType) {
- if (argType == INT_FAMILY ||
- argType == DATE ||
- argType == INTERVAL_YEAR_MONTH
- ) {
- return "Long";
- } else if (argType == TIMESTAMP ||
- argType == INTERVAL_DAY_TIME) {
- return "Timestamp";
- } else if (argType == FLOAT_FAMILY) {
- return "Double";
- } else if (argType == DECIMAL) {
- return "Decimal";
- } else if (argType == STRING ||
- argType == CHAR ||
- argType == VARCHAR ||
- argType == BINARY) {
- return "String";
- } else {
- return "None";
- }
- }
-
- public static String getVectorColumnSimpleName(String hiveTypeName) {
- ArgumentType argType = fromHiveTypeName(hiveTypeName);
- return getVectorColumnSimpleName(argType);
- }
}
public enum InputExpressionType {
http://git-wip-us.apache.org/repos/asf/hive/blob/25aaf7db/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapperBatch.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapperBatch.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapperBatch.java
index 0e6f8c5..689d3c3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapperBatch.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapperBatch.java
@@ -1037,7 +1037,7 @@ public class VectorHashKeyWrapperBatch extends VectorColumnSetInfo {
kw.getByteLength(columnTypeSpecificIndex));
break;
case DECIMAL:
- ((DecimalColumnVector) colVector).vector[batchIndex].set(
+ ((DecimalColumnVector) colVector).set(batchIndex,
kw.getDecimal(columnTypeSpecificIndex));
break;
case TIMESTAMP:
http://git-wip-us.apache.org/repos/asf/hive/blob/25aaf7db/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 491a6b1..45ceffc 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
@@ -133,6 +133,7 @@ public class VectorizationContext {
}
private HiveVectorAdaptorUsageMode hiveVectorAdaptorUsageMode;
+ private boolean testVectorAdaptorOverride;
public enum HiveVectorIfStmtMode {
ADAPTOR,
@@ -158,6 +159,8 @@ public class VectorizationContext {
private void setHiveConfVars(HiveConf hiveConf) {
hiveVectorAdaptorUsageMode = HiveVectorAdaptorUsageMode.getHiveConfValue(hiveConf);
+ testVectorAdaptorOverride =
+ HiveConf.getBoolVar(hiveConf, ConfVars.HIVE_TEST_VECTOR_ADAPTOR_OVERRIDE);
hiveVectorIfStmtMode = HiveVectorIfStmtMode.getHiveConfValue(hiveConf);
this.reuseScratchColumns =
HiveConf.getBoolVar(hiveConf, ConfVars.HIVE_VECTORIZATION_TESTING_REUSE_SCRATCH_COLUMNS);
@@ -171,8 +174,11 @@ public class VectorizationContext {
private void copyHiveConfVars(VectorizationContext vContextEnvironment) {
hiveVectorAdaptorUsageMode = vContextEnvironment.hiveVectorAdaptorUsageMode;
+ testVectorAdaptorOverride = vContextEnvironment.testVectorAdaptorOverride;
hiveVectorIfStmtMode = vContextEnvironment.hiveVectorIfStmtMode;
this.reuseScratchColumns = vContextEnvironment.reuseScratchColumns;
+ useCheckedVectorExpressions = vContextEnvironment.useCheckedVectorExpressions;
+ adaptorSuppressEvaluateExceptions = vContextEnvironment.adaptorSuppressEvaluateExceptions;
this.ocm.setReuseColumns(reuseScratchColumns);
}
@@ -801,8 +807,12 @@ public class VectorizationContext {
// Note: this is a no-op for custom UDFs
List<ExprNodeDesc> childExpressions = getChildExpressionsWithImplicitCast(expr.getGenericUDF(),
exprDesc.getChildren(), exprDesc.getTypeInfo());
- ve = getGenericUdfVectorExpression(expr.getGenericUDF(),
- childExpressions, mode, exprDesc.getTypeInfo());
+
+ // Are we forcing the usage of VectorUDFAdaptor for test purposes?
+ if (!testVectorAdaptorOverride) {
+ ve = getGenericUdfVectorExpression(expr.getGenericUDF(),
+ childExpressions, mode, exprDesc.getTypeInfo());
+ }
if (ve == null) {
// Ok, no vectorized class available. No problem -- try to use the VectorUDFAdaptor
// when configured.
@@ -1104,7 +1114,7 @@ public class VectorizationContext {
return HiveDecimalUtils.getPrecisionForType(typeInfo);
}
- private GenericUDF getGenericUDFForCast(TypeInfo castType) throws HiveException {
+ public static GenericUDF getGenericUDFForCast(TypeInfo castType) throws HiveException {
UDF udfClass = null;
GenericUDF genericUdf = null;
switch (((PrimitiveTypeInfo) castType).getPrimitiveCategory()) {
@@ -1165,8 +1175,10 @@ public class VectorizationContext {
if (udfClass == null) {
throw new HiveException("Could not add implicit cast for type "+castType.getTypeName());
}
- genericUdf = new GenericUDFBridge();
- ((GenericUDFBridge) genericUdf).setUdfClassName(udfClass.getClass().getName());
+ GenericUDFBridge genericUDFBridge = new GenericUDFBridge();
+ genericUDFBridge.setUdfClassName(udfClass.getClass().getName());
+ genericUDFBridge.setUdfName(udfClass.getClass().getSimpleName());
+ genericUdf = genericUDFBridge;
}
if (genericUdf instanceof SettableUDF) {
((SettableUDF) genericUdf).setTypeInfo(castType);
@@ -2736,7 +2748,9 @@ public class VectorizationContext {
}
if (isIntFamily(inputType)) {
return createVectorExpression(CastLongToDecimal.class, childExpr, VectorExpressionDescriptor.Mode.PROJECTION, returnType);
- } else if (isFloatFamily(inputType)) {
+ } else if (inputType.equals("float")) {
+ return createVectorExpression(CastFloatToDecimal.class, childExpr, VectorExpressionDescriptor.Mode.PROJECTION, returnType);
+ } else if (inputType.equals("double")) {
return createVectorExpression(CastDoubleToDecimal.class, childExpr, VectorExpressionDescriptor.Mode.PROJECTION, returnType);
} else if (decimalTypePattern.matcher(inputType).matches()) {
if (child instanceof ExprNodeColumnDesc) {
http://git-wip-us.apache.org/repos/asf/hive/blob/25aaf7db/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedBatchUtil.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedBatchUtil.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedBatchUtil.java
index d92ec32..d51d44a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedBatchUtil.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedBatchUtil.java
@@ -880,6 +880,10 @@ public class VectorizedBatchUtil {
return newBatch;
}
+ public static Writable getPrimitiveWritable(TypeInfo typeInfo) {
+ return getPrimitiveWritable(((PrimitiveTypeInfo) typeInfo).getPrimitiveCategory());
+ }
+
public static Writable getPrimitiveWritable(PrimitiveCategory primitiveCategory) {
switch (primitiveCategory) {
case VOID:
http://git-wip-us.apache.org/repos/asf/hive/blob/25aaf7db/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDateToBoolean.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDateToBoolean.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDateToBoolean.java
new file mode 100644
index 0000000..117e814
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDateToBoolean.java
@@ -0,0 +1,61 @@
+/*
+ * 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.exec.vector.expressions;
+
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+
+/*
+ * Comment from BooleanWritable evaluate(DateWritable d)
+ * // date value to boolean doesn't make any sense.
+ * So, we always set the output to NULL.
+ */
+public class CastDateToBoolean extends NullVectorExpression {
+ private static final long serialVersionUID = 1L;
+
+ private final int colNum;
+
+ public CastDateToBoolean(int colNum, int outputColumnNum) {
+ super(outputColumnNum);
+ this.colNum = colNum;
+ }
+
+ public CastDateToBoolean() {
+ super();
+
+ // Dummy final assignments.
+ colNum = -1;
+ }
+
+ @Override
+ public String vectorExpressionParameters() {
+ return getColumnParamString(0, colNum);
+ }
+
+ @Override
+ public VectorExpressionDescriptor.Descriptor getDescriptor() {
+ return (new VectorExpressionDescriptor.Builder())
+ .setMode(
+ VectorExpressionDescriptor.Mode.PROJECTION)
+ .setNumArguments(1)
+ .setArgumentTypes(
+ VectorExpressionDescriptor.ArgumentType.getType("date"))
+ .setInputExpressionTypes(
+ VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
+ }
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/25aaf7db/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToDecimal.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToDecimal.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToDecimal.java
index 5e0d570..bcf55cd 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToDecimal.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToDecimal.java
@@ -57,7 +57,7 @@ public class CastDecimalToDecimal extends VectorExpression {
*/
protected void convert(DecimalColumnVector outputColVector, DecimalColumnVector inputColVector, int i) {
// The set routine enforces precision and scale.
- outputColVector.vector[i].set(inputColVector.vector[i]);
+ outputColVector.set(i, inputColVector.vector[i]);
}
/**
http://git-wip-us.apache.org/repos/asf/hive/blob/25aaf7db/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDoubleToDecimal.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDoubleToDecimal.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDoubleToDecimal.java
index 4619724..b6a7a26 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDoubleToDecimal.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDoubleToDecimal.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.ql.exec.vector.expressions;
import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
/**
@@ -41,9 +42,21 @@ public class CastDoubleToDecimal extends FuncDoubleToDecimal {
protected void func(DecimalColumnVector outV, DoubleColumnVector inV, int i) {
HiveDecimalWritable decWritable = outV.vector[i];
decWritable.setFromDouble(inV.vector[i]);
- if (!decWritable.isSet()) {
+ if (!decWritable.mutateEnforcePrecisionScale(outV.precision, outV.scale)) {
outV.isNull[i] = true;
outV.noNulls = false;
}
}
+
+ @Override
+ public VectorExpressionDescriptor.Descriptor getDescriptor() {
+ VectorExpressionDescriptor.Builder b = new VectorExpressionDescriptor.Builder();
+ b.setMode(VectorExpressionDescriptor.Mode.PROJECTION)
+ .setNumArguments(1)
+ .setArgumentTypes(
+ VectorExpressionDescriptor.ArgumentType.FLOAT)
+ .setInputExpressionTypes(
+ VectorExpressionDescriptor.InputExpressionType.COLUMN);
+ return b.build();
+ }
}
http://git-wip-us.apache.org/repos/asf/hive/blob/25aaf7db/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastFloatToDecimal.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastFloatToDecimal.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastFloatToDecimal.java
new file mode 100644
index 0000000..0462334
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastFloatToDecimal.java
@@ -0,0 +1,65 @@
+/*
+ * 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.exec.vector.expressions;
+
+import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+
+/**
+ * Cast input float to a decimal. Get target value scale from output column vector.
+ */
+public class CastFloatToDecimal extends FuncDoubleToDecimal {
+
+ private static final long serialVersionUID = 1L;
+
+ public CastFloatToDecimal() {
+ super();
+ }
+
+ public CastFloatToDecimal(int inputColumn, int outputColumnNum) {
+ super(inputColumn, outputColumnNum);
+ }
+
+ @Override
+ protected void func(DecimalColumnVector outV, DoubleColumnVector inV, int i) {
+ HiveDecimalWritable decWritable = outV.vector[i];
+
+ // TEMPORARY: In order to avoid a new version of storage-api, do the conversion here...
+ byte[] floatBytes = Float.toString((float) inV.vector[i]).getBytes();
+ decWritable.setFromBytes(floatBytes, 0, floatBytes.length);
+ if (!decWritable.mutateEnforcePrecisionScale(outV.precision, outV.scale)) {
+ outV.isNull[i] = true;
+ outV.noNulls = false;
+ }
+ }
+
+ @Override
+ public VectorExpressionDescriptor.Descriptor getDescriptor() {
+ VectorExpressionDescriptor.Builder b = new VectorExpressionDescriptor.Builder();
+ b.setMode(VectorExpressionDescriptor.Mode.PROJECTION)
+ .setNumArguments(1)
+ .setArgumentTypes(
+ VectorExpressionDescriptor.ArgumentType.FLOAT)
+ .setInputExpressionTypes(
+ VectorExpressionDescriptor.InputExpressionType.COLUMN);
+ return b.build();
+ }
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/25aaf7db/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastLongToDecimal.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastLongToDecimal.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastLongToDecimal.java
index f8edbd9..fa88e3f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastLongToDecimal.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastLongToDecimal.java
@@ -41,6 +41,6 @@ public class CastLongToDecimal extends FuncLongToDecimal {
@Override
protected void func(DecimalColumnVector outV, LongColumnVector inV, int i) {
- outV.vector[i].set(HiveDecimal.create(inV.vector[i]));
+ outV.set(i, HiveDecimal.create(inV.vector[i]));
}
}
http://git-wip-us.apache.org/repos/asf/hive/blob/25aaf7db/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToDecimal.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToDecimal.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToDecimal.java
index d8d7dae..7dc322e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToDecimal.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToDecimal.java
@@ -63,7 +63,7 @@ public class CastStringToDecimal extends VectorExpression {
* making a new string.
*/
s = new String(inputColVector.vector[i], inputColVector.start[i], inputColVector.length[i], "UTF-8");
- outputColVector.vector[i].set(HiveDecimal.create(s));
+ outputColVector.set(i, HiveDecimal.create(s));
} catch (Exception e) {
// for any exception in conversion to decimal, produce NULL
http://git-wip-us.apache.org/repos/asf/hive/blob/25aaf7db/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastTimestampToLong.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastTimestampToLong.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastTimestampToLong.java
index 42e005e..3f5f25d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastTimestampToLong.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastTimestampToLong.java
@@ -24,12 +24,16 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
import org.apache.hadoop.hive.ql.exec.vector.expressions.MathExpr;
import org.apache.hadoop.hive.ql.exec.vector.*;
import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
+import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
public class CastTimestampToLong extends VectorExpression {
private static final long serialVersionUID = 1L;
private int colNum;
+ private transient PrimitiveCategory integerPrimitiveCategory;
+
public CastTimestampToLong(int colNum, int outputColumnNum) {
super(outputColumnNum);
this.colNum = colNum;
@@ -40,6 +44,41 @@ public class CastTimestampToLong extends VectorExpression {
}
@Override
+ public void transientInit() throws HiveException {
+ integerPrimitiveCategory = ((PrimitiveTypeInfo) outputTypeInfo).getPrimitiveCategory();
+ }
+
+ private void setIntegerFromTimestamp(TimestampColumnVector inputColVector,
+ LongColumnVector outputColVector, int batchIndex) {
+
+ final long longValue = inputColVector.getTimestampAsLong(batchIndex);
+
+ boolean isInRange;
+ switch (integerPrimitiveCategory) {
+ case BYTE:
+ isInRange = ((byte) longValue) == longValue;
+ break;
+ case SHORT:
+ isInRange = ((short) longValue) == longValue;
+ break;
+ case INT:
+ isInRange = ((int) longValue) == longValue;
+ break;
+ case LONG:
+ isInRange = true;
+ break;
+ default:
+ throw new RuntimeException("Unexpected integer primitive category " + integerPrimitiveCategory);
+ }
+ if (isInRange) {
+ outputColVector.vector[batchIndex] = longValue;
+ } else {
+ outputColVector.isNull[batchIndex] = true;
+ outputColVector.noNulls = false;
+ }
+ }
+
+ @Override
public void evaluate(VectorizedRowBatch batch) throws HiveException {
if (childExpressions != null) {
@@ -52,7 +91,6 @@ public class CastTimestampToLong extends VectorExpression {
boolean[] inputIsNull = inputColVector.isNull;
boolean[] outputIsNull = outputColVector.isNull;
int n = batch.size;
- long[] outputVector = outputColVector.vector;
// return immediately if batch is empty
if (n == 0) {
@@ -65,7 +103,7 @@ public class CastTimestampToLong extends VectorExpression {
if (inputColVector.isRepeating) {
if (inputColVector.noNulls || !inputIsNull[0]) {
outputIsNull[0] = false;
- outputVector[0] = inputColVector.getTimestampAsLong(0);
+ setIntegerFromTimestamp(inputColVector, outputColVector, 0);
} else {
outputIsNull[0] = true;
outputColVector.noNulls = false;
@@ -84,12 +122,12 @@ public class CastTimestampToLong extends VectorExpression {
final int i = sel[j];
// Set isNull before call in case it changes it mind.
outputIsNull[i] = false;
- outputVector[i] = inputColVector.getTimestampAsLong(i);
+ setIntegerFromTimestamp(inputColVector, outputColVector, i);
}
} else {
for(int j = 0; j != n; j++) {
final int i = sel[j];
- outputVector[i] = inputColVector.getTimestampAsLong(i);
+ setIntegerFromTimestamp(inputColVector, outputColVector, i);
}
}
} else {
@@ -101,7 +139,7 @@ public class CastTimestampToLong extends VectorExpression {
outputColVector.noNulls = true;
}
for(int i = 0; i != n; i++) {
- outputVector[i] = inputColVector.getTimestampAsLong(i);
+ setIntegerFromTimestamp(inputColVector, outputColVector, i);
}
}
} else /* there are NULLs in the inputColVector */ {
@@ -114,20 +152,20 @@ public class CastTimestampToLong extends VectorExpression {
for(int j = 0; j != n; j++) {
int i = sel[j];
if (!inputIsNull[i]) {
- inputIsNull[i] = false;
- outputVector[i] = inputColVector.getTimestampAsLong(i);
+ outputIsNull[i] = false;
+ setIntegerFromTimestamp(inputColVector, outputColVector, i);
} else {
- inputIsNull[i] = true;
+ outputIsNull[i] = true;
outputColVector.noNulls = false;
}
}
} else {
for(int i = 0; i != n; i++) {
if (!inputIsNull[i]) {
- inputIsNull[i] = false;
- outputVector[i] = inputColVector.getTimestampAsLong(i);
+ outputIsNull[i] = false;
+ setIntegerFromTimestamp(inputColVector, outputColVector, i);
} else {
- inputIsNull[i] = true;
+ outputIsNull[i] = true;
outputColVector.noNulls = false;
}
}
http://git-wip-us.apache.org/repos/asf/hive/blob/25aaf7db/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/NullVectorExpression.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/NullVectorExpression.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/NullVectorExpression.java
new file mode 100644
index 0000000..b7bfe1e
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/NullVectorExpression.java
@@ -0,0 +1,56 @@
+/*
+ * 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.exec.vector.expressions;
+
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.*;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor.Descriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+
+public class NullVectorExpression extends VectorExpression {
+ private static final long serialVersionUID = 1L;
+
+ public NullVectorExpression(int outputColumnNum) {
+ super(outputColumnNum);
+ }
+
+ public NullVectorExpression() {
+ super();
+ }
+
+
+ @Override
+ public String vectorExpressionParameters() {
+ return null;
+ }
+
+ @Override
+ public void evaluate(VectorizedRowBatch batch) throws HiveException {
+ ColumnVector colVector = batch.cols[outputColumnNum];
+ colVector.isNull[0] = true;
+ colVector.noNulls = false;
+ colVector.isRepeating = true;
+ }
+
+ @Override
+ public Descriptor getDescriptor() {
+ // Not applicable.
+ return null;
+ }
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/25aaf7db/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFSumDecimal.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFSumDecimal.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFSumDecimal.java
index 95703b0..315b72b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFSumDecimal.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFSumDecimal.java
@@ -460,6 +460,6 @@ public class VectorUDAFSumDecimal extends VectorAggregateExpression {
}
outputColVector.isNull[batchIndex] = false;
- outputColVector.vector[batchIndex].set(myagg.sum);
+ outputColVector.set(batchIndex, myagg.sum);
}
}
http://git-wip-us.apache.org/repos/asf/hive/blob/25aaf7db/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFSumDecimal64ToDecimal.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFSumDecimal64ToDecimal.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFSumDecimal64ToDecimal.java
index d091f3f..117611e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFSumDecimal64ToDecimal.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFSumDecimal64ToDecimal.java
@@ -516,6 +516,6 @@ public class VectorUDAFSumDecimal64ToDecimal extends VectorAggregateExpression {
}
outputColVector.isNull[batchIndex] = false;
- outputColVector.vector[batchIndex].set(myagg.regularDecimalSum);
+ outputColVector.set(batchIndex, myagg.regularDecimalSum);
}
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hive/blob/25aaf7db/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDecimalFirstValue.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDecimalFirstValue.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDecimalFirstValue.java
index ce118bc..dc037ae 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDecimalFirstValue.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFEvaluatorDecimalFirstValue.java
@@ -102,7 +102,7 @@ public class VectorPTFEvaluatorDecimalFirstValue extends VectorPTFEvaluatorBase
outputColVector.isNull[0] = true;
} else {
outputColVector.isNull[0] = false;
- outputColVector.vector[0].set(firstValue);
+ outputColVector.set(0, firstValue);
}
}
http://git-wip-us.apache.org/repos/asf/hive/blob/25aaf7db/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFGroupBatches.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFGroupBatches.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFGroupBatches.java
index 573910e..a39da0d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFGroupBatches.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ptf/VectorPTFGroupBatches.java
@@ -206,7 +206,7 @@ public class VectorPTFGroupBatches {
((DoubleColumnVector) outputColVector).vector[0] = evaluator.getDoubleGroupResult();
break;
case DECIMAL:
- ((DecimalColumnVector) outputColVector).vector[0].set(evaluator.getDecimalGroupResult());
+ ((DecimalColumnVector) outputColVector).set(0, evaluator.getDecimalGroupResult());
break;
default:
throw new RuntimeException("Unexpected column vector type " + evaluator.getResultColumnVectorType());
http://git-wip-us.apache.org/repos/asf/hive/blob/25aaf7db/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeGenericFuncDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeGenericFuncDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeGenericFuncDesc.java
index 961eea2..002aef6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeGenericFuncDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeGenericFuncDesc.java
@@ -34,6 +34,7 @@ import com.google.common.collect.ImmutableSortedMultiset;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hive.conf.HiveConf.StrictChecks;
import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
+import org.apache.hadoop.hive.ql.exec.UDF;
import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
import org.apache.hadoop.hive.ql.exec.Utilities;
import org.apache.hadoop.hive.ql.session.SessionState;
@@ -144,7 +145,9 @@ public class ExprNodeGenericFuncDesc extends ExprNodeDesc implements
if (genericUDF instanceof GenericUDFBridge) {
GenericUDFBridge genericUDFBridge = (GenericUDFBridge) genericUDF;
sb.append(" ==> ");
- sb.append(genericUDFBridge.getUdfName());
+ String udfName = genericUDFBridge.getUdfName();
+ Class<? extends UDF> udfClass = genericUDFBridge.getUdfClass();
+ sb.append(udfName != null ? udfName : (udfClass != null ? udfClass.getSimpleName() : "null"));
sb.append(" ");
}
sb.append("(");
http://git-wip-us.apache.org/repos/asf/hive/blob/25aaf7db/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToBoolean.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToBoolean.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToBoolean.java
index d7d8bcc..3ac7a06 100755
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToBoolean.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToBoolean.java
@@ -27,7 +27,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.CastStringToBoolean;
import org.apache.hadoop.hive.ql.exec.vector.expressions.CastStringToLong;
import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.CastDoubleToBooleanViaDoubleToLong;
import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.CastLongToBooleanViaLongToLong;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.CastDateToBooleanViaLongToLong;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.CastDateToBoolean;
import org.apache.hadoop.hive.ql.exec.vector.expressions.CastTimestampToBoolean;
import org.apache.hadoop.hive.serde2.io.ByteWritable;
import org.apache.hadoop.hive.serde2.io.DateWritable;
@@ -48,7 +48,7 @@ import org.apache.hadoop.io.Text;
*
*/
@VectorizedExpressions({CastLongToBooleanViaLongToLong.class,
- CastDateToBooleanViaLongToLong.class, CastTimestampToBoolean.class, CastStringToBoolean.class,
+ CastDateToBoolean.class, CastTimestampToBoolean.class, CastStringToBoolean.class,
CastDoubleToBooleanViaDoubleToLong.class, CastDecimalToBoolean.class, CastStringToLong.class})
public class UDFToBoolean extends UDF {
private final BooleanWritable booleanWritable = new BooleanWritable();
http://git-wip-us.apache.org/repos/asf/hive/blob/25aaf7db/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToByte.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToByte.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToByte.java
index 8c6629e..1128b32 100755
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToByte.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToByte.java
@@ -187,7 +187,12 @@ public class UDFToByte extends UDF {
if (i == null) {
return null;
} else {
- byteWritable.set((byte)i.getSeconds());
+ final long longValue = i.getSeconds();
+ final byte byteValue = (byte) longValue;
+ if (byteValue != longValue) {
+ return null;
+ }
+ byteWritable.set(byteValue);
return byteWritable;
}
}
http://git-wip-us.apache.org/repos/asf/hive/blob/25aaf7db/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToInteger.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToInteger.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToInteger.java
index 9540449..748a688 100755
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToInteger.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToInteger.java
@@ -197,7 +197,12 @@ public class UDFToInteger extends UDF {
if (i == null) {
return null;
} else {
- intWritable.set((int) i.getSeconds());
+ final long longValue = i.getSeconds();
+ final int intValue = (int) longValue;
+ if (intValue != longValue) {
+ return null;
+ }
+ intWritable.set(intValue);
return intWritable;
}
}
http://git-wip-us.apache.org/repos/asf/hive/blob/25aaf7db/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToShort.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToShort.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToShort.java
index 94bbe82..e003ff3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToShort.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToShort.java
@@ -189,7 +189,12 @@ public class UDFToShort extends UDF {
if (i == null) {
return null;
} else {
- shortWritable.set((short) i.getSeconds());
+ final long longValue = i.getSeconds();
+ final short shortValue = (short) longValue;
+ if (shortValue != longValue) {
+ return null;
+ }
+ shortWritable.set(shortValue);
return shortWritable;
}
}
http://git-wip-us.apache.org/repos/asf/hive/blob/25aaf7db/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/VectorRandomRowSource.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/VectorRandomRowSource.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/VectorRandomRowSource.java
index fa5c775..7877532 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/VectorRandomRowSource.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/VectorRandomRowSource.java
@@ -222,21 +222,23 @@ public class VectorRandomRowSource {
"map"
};
- private String getRandomTypeName(SupportedTypes supportedTypes, Set<String> allowedTypeNameSet) {
+ private static String getRandomTypeName(Random random, SupportedTypes supportedTypes,
+ Set<String> allowedTypeNameSet) {
+
String typeName = null;
do {
- if (r.nextInt(10 ) != 0) {
- typeName = possibleHivePrimitiveTypeNames[r.nextInt(possibleHivePrimitiveTypeNames.length)];
+ if (random.nextInt(10 ) != 0) {
+ typeName = possibleHivePrimitiveTypeNames[random.nextInt(possibleHivePrimitiveTypeNames.length)];
} else {
switch (supportedTypes) {
case PRIMITIVES:
- typeName = possibleHivePrimitiveTypeNames[r.nextInt(possibleHivePrimitiveTypeNames.length)];
+ typeName = possibleHivePrimitiveTypeNames[random.nextInt(possibleHivePrimitiveTypeNames.length)];
break;
case ALL_EXCEPT_MAP:
- typeName = possibleHiveComplexTypeNames[r.nextInt(possibleHiveComplexTypeNames.length - 1)];
+ typeName = possibleHiveComplexTypeNames[random.nextInt(possibleHiveComplexTypeNames.length - 1)];
break;
case ALL:
- typeName = possibleHiveComplexTypeNames[r.nextInt(possibleHiveComplexTypeNames.length)];
+ typeName = possibleHiveComplexTypeNames[random.nextInt(possibleHiveComplexTypeNames.length)];
break;
}
}
@@ -244,17 +246,22 @@ public class VectorRandomRowSource {
return typeName;
}
- private String getDecoratedTypeName(String typeName, SupportedTypes supportedTypes,
- Set<String> allowedTypeNameSet, int depth, int maxDepth) {
+ public static String getDecoratedTypeName(Random random, String typeName) {
+ return getDecoratedTypeName(random, typeName, null, null, 0, 1);
+ }
+
+ private static String getDecoratedTypeName(Random random, String typeName,
+ SupportedTypes supportedTypes, Set<String> allowedTypeNameSet, int depth, int maxDepth) {
+
depth++;
if (depth < maxDepth) {
supportedTypes = SupportedTypes.PRIMITIVES;
}
if (typeName.equals("char")) {
- final int maxLength = 1 + r.nextInt(100);
+ final int maxLength = 1 + random.nextInt(100);
typeName = String.format("char(%d)", maxLength);
} else if (typeName.equals("varchar")) {
- final int maxLength = 1 + r.nextInt(100);
+ final int maxLength = 1 + random.nextInt(100);
typeName = String.format("varchar(%d)", maxLength);
} else if (typeName.equals("decimal")) {
typeName =
@@ -263,26 +270,34 @@ public class VectorRandomRowSource {
HiveDecimal.SYSTEM_DEFAULT_PRECISION,
HiveDecimal.SYSTEM_DEFAULT_SCALE);
} else if (typeName.equals("array")) {
- String elementTypeName = getRandomTypeName(supportedTypes, allowedTypeNameSet);
+ String elementTypeName = getRandomTypeName(random, supportedTypes, allowedTypeNameSet);
elementTypeName =
- getDecoratedTypeName(elementTypeName, supportedTypes, allowedTypeNameSet, depth, maxDepth);
+ getDecoratedTypeName(random, elementTypeName, supportedTypes, allowedTypeNameSet, depth, maxDepth);
typeName = String.format("array<%s>", elementTypeName);
} else if (typeName.equals("map")) {
- String keyTypeName = getRandomTypeName(SupportedTypes.PRIMITIVES, allowedTypeNameSet);
+ String keyTypeName =
+ getRandomTypeName(
+ random, SupportedTypes.PRIMITIVES, allowedTypeNameSet);
keyTypeName =
- getDecoratedTypeName(keyTypeName, supportedTypes, allowedTypeNameSet, depth, maxDepth);
- String valueTypeName = getRandomTypeName(supportedTypes, allowedTypeNameSet);
+ getDecoratedTypeName(
+ random, keyTypeName, supportedTypes, allowedTypeNameSet, depth, maxDepth);
+ String valueTypeName =
+ getRandomTypeName(
+ random, supportedTypes, allowedTypeNameSet);
valueTypeName =
- getDecoratedTypeName(valueTypeName, supportedTypes, allowedTypeNameSet, depth, maxDepth);
+ getDecoratedTypeName(
+ random, valueTypeName, supportedTypes, allowedTypeNameSet, depth, maxDepth);
typeName = String.format("map<%s,%s>", keyTypeName, valueTypeName);
} else if (typeName.equals("struct")) {
- final int fieldCount = 1 + r.nextInt(10);
+ final int fieldCount = 1 + random.nextInt(10);
final StringBuilder sb = new StringBuilder();
for (int i = 0; i < fieldCount; i++) {
- String fieldTypeName = getRandomTypeName(supportedTypes, allowedTypeNameSet);
+ String fieldTypeName =
+ getRandomTypeName(
+ random, supportedTypes, allowedTypeNameSet);
fieldTypeName =
getDecoratedTypeName(
- fieldTypeName, supportedTypes, allowedTypeNameSet, depth, maxDepth);
+ random, fieldTypeName, supportedTypes, allowedTypeNameSet, depth, maxDepth);
if (i > 0) {
sb.append(",");
}
@@ -294,13 +309,15 @@ public class VectorRandomRowSource {
typeName = String.format("struct<%s>", sb.toString());
} else if (typeName.equals("struct") ||
typeName.equals("uniontype")) {
- final int fieldCount = 1 + r.nextInt(10);
+ final int fieldCount = 1 + random.nextInt(10);
final StringBuilder sb = new StringBuilder();
for (int i = 0; i < fieldCount; i++) {
- String fieldTypeName = getRandomTypeName(supportedTypes, allowedTypeNameSet);
+ String fieldTypeName =
+ getRandomTypeName(
+ random, supportedTypes, allowedTypeNameSet);
fieldTypeName =
getDecoratedTypeName(
- fieldTypeName, supportedTypes, allowedTypeNameSet, depth, maxDepth);
+ random, fieldTypeName, supportedTypes, allowedTypeNameSet, depth, maxDepth);
if (i > 0) {
sb.append(",");
}
@@ -311,6 +328,11 @@ public class VectorRandomRowSource {
return typeName;
}
+ private String getDecoratedTypeName(String typeName,
+ SupportedTypes supportedTypes, Set<String> allowedTypeNameSet, int depth, int maxDepth) {
+ return getDecoratedTypeName(r, typeName, supportedTypes, allowedTypeNameSet, depth, maxDepth);
+ }
+
private ObjectInspector getObjectInspector(TypeInfo typeInfo) {
return getObjectInspector(typeInfo, DataTypePhysicalVariation.NONE);
}
@@ -454,7 +476,7 @@ public class VectorRandomRowSource {
typeName = explicitTypeNameList.get(c);
dataTypePhysicalVariation = explicitDataTypePhysicalVariationList.get(c);
} else if (onlyOne || allowedTypeNameSet != null) {
- typeName = getRandomTypeName(supportedTypes, allowedTypeNameSet);
+ typeName = getRandomTypeName(r, supportedTypes, allowedTypeNameSet);
} else {
int typeNum;
if (allTypes) {
http://git-wip-us.apache.org/repos/asf/hive/blob/25aaf7db/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorCastStatement.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorCastStatement.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorCastStatement.java
new file mode 100644
index 0000000..0e300cf
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorCastStatement.java
@@ -0,0 +1,502 @@
+/*
+ * 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.exec.vector.expressions;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.hadoop.hive.common.type.DataTypePhysicalVariation;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator;
+import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluatorFactory;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExtractRow;
+import org.apache.hadoop.hive.ql.exec.vector.VectorRandomBatchSource;
+import org.apache.hadoop.hive.ql.exec.vector.VectorRandomRowSource;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizationContext;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatchCtx;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.IdentityExpression;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.metadata.VirtualColumn;
+import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFIf;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFWhen;
+import org.apache.hadoop.hive.serde2.io.DoubleWritable;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
+import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
+import org.apache.hadoop.io.LongWritable;
+
+import junit.framework.Assert;
+
+import org.junit.Ignore;
+import org.junit.Test;
+
+public class TestVectorCastStatement {
+
+ @Test
+ public void testBoolean() throws Exception {
+ Random random = new Random(12882);
+
+ doIfTests(random, "boolean");
+ }
+
+ @Test
+ public void testTinyInt() throws Exception {
+ Random random = new Random(5371);
+
+ doIfTests(random, "tinyint");
+ }
+
+ @Test
+ public void testSmallInt() throws Exception {
+ Random random = new Random(2772);
+
+ doIfTests(random, "smallint");
+ }
+
+ @Test
+ public void testInt() throws Exception {
+ Random random = new Random(12882);
+
+ doIfTests(random, "int");
+ }
+
+ @Test
+ public void testBigInt() throws Exception {
+ Random random = new Random(12882);
+
+ doIfTests(random, "bigint");
+ }
+
+ @Test
+ public void testString() throws Exception {
+ Random random = new Random(12882);
+
+ doIfTests(random, "string");
+ }
+
+ @Test
+ public void testTimestamp() throws Exception {
+ Random random = new Random(12882);
+
+ doIfTests(random, "timestamp");
+ }
+
+ @Test
+ public void testDate() throws Exception {
+ Random random = new Random(12882);
+
+ doIfTests(random, "date");
+ }
+
+ @Test
+ public void testFloat() throws Exception {
+ Random random = new Random(7322);
+
+ doIfTests(random, "float");
+ }
+
+ @Test
+ public void testDouble() throws Exception {
+ Random random = new Random(12882);
+
+ doIfTests(random, "double");
+ }
+
+ @Test
+ public void testChar() throws Exception {
+ Random random = new Random(12882);
+
+ doIfTests(random, "char(10)");
+ }
+
+ @Test
+ public void testVarchar() throws Exception {
+ Random random = new Random(12882);
+
+ doIfTests(random, "varchar(15)");
+ }
+
+ @Test
+ public void testBinary() throws Exception {
+ Random random = new Random(12882);
+
+ doIfTests(random, "binary");
+ }
+
+ @Test
+ public void testDecimal() throws Exception {
+ Random random = new Random(9300);
+
+ doIfTests(random, "decimal(38,18)");
+ doIfTests(random, "decimal(38,0)");
+ doIfTests(random, "decimal(20,8)");
+ doIfTests(random, "decimal(10,4)");
+ }
+
+ public enum CastStmtTestMode {
+ ROW_MODE,
+ ADAPTOR,
+ VECTOR_EXPRESSION;
+
+ static final int count = values().length;
+ }
+
+ private void doIfTests(Random random, String typeName)
+ throws Exception {
+ doIfTests(random, typeName, DataTypePhysicalVariation.NONE);
+ }
+
+ private void doIfTests(Random random, String typeName,
+ DataTypePhysicalVariation dataTypePhysicalVariation)
+ throws Exception {
+
+ TypeInfo typeInfo = TypeInfoUtils.getTypeInfoFromTypeString(typeName);
+ PrimitiveCategory primitiveCategory = ((PrimitiveTypeInfo) typeInfo).getPrimitiveCategory();
+
+ for (PrimitiveCategory targetPrimitiveCategory : PrimitiveCategory.values()) {
+
+ if (targetPrimitiveCategory == PrimitiveCategory.VOID ||
+ targetPrimitiveCategory == PrimitiveCategory.INTERVAL_YEAR_MONTH ||
+ targetPrimitiveCategory == PrimitiveCategory.INTERVAL_DAY_TIME ||
+ targetPrimitiveCategory == PrimitiveCategory.TIMESTAMPLOCALTZ ||
+ targetPrimitiveCategory == PrimitiveCategory.UNKNOWN) {
+ continue;
+ }
+
+ // BINARY conversions supported by GenericUDFDecimal, GenericUDFTimestamp.
+ if (primitiveCategory == PrimitiveCategory.BINARY) {
+ if (targetPrimitiveCategory == PrimitiveCategory.DECIMAL ||
+ targetPrimitiveCategory == PrimitiveCategory.TIMESTAMP) {
+ continue;
+ }
+ }
+
+ // DATE conversions supported by GenericUDFDecimal.
+ if (primitiveCategory == PrimitiveCategory.DATE) {
+ if (targetPrimitiveCategory == PrimitiveCategory.DECIMAL) {
+ continue;
+ }
+ }
+
+ if (primitiveCategory == targetPrimitiveCategory) {
+ if (primitiveCategory != PrimitiveCategory.CHAR &&
+ primitiveCategory != PrimitiveCategory.VARCHAR &&
+ primitiveCategory != PrimitiveCategory.DECIMAL) {
+ continue;
+ }
+ }
+
+ doIfTestOneCast(random, typeName, dataTypePhysicalVariation, targetPrimitiveCategory);
+ }
+ }
+
+ private void doIfTestOneCast(Random random, String typeName,
+ DataTypePhysicalVariation dataTypePhysicalVariation,
+ PrimitiveCategory targetPrimitiveCategory)
+ throws Exception {
+
+ TypeInfo typeInfo = TypeInfoUtils.getTypeInfoFromTypeString(typeName);
+
+ boolean isDecimal64 = (dataTypePhysicalVariation == DataTypePhysicalVariation.DECIMAL_64);
+ final int decimal64Scale =
+ (isDecimal64 ? ((DecimalTypeInfo) typeInfo).getScale() : 0);
+
+ List<String> explicitTypeNameList = new ArrayList<String>();
+ List<DataTypePhysicalVariation> explicitDataTypePhysicalVariationList = new ArrayList<DataTypePhysicalVariation>();
+ explicitTypeNameList.add(typeName);
+ explicitDataTypePhysicalVariationList.add(dataTypePhysicalVariation);
+
+ VectorRandomRowSource rowSource = new VectorRandomRowSource();
+
+ rowSource.initExplicitSchema(
+ random, explicitTypeNameList, /* maxComplexDepth */ 0, /* allowNull */ true,
+ explicitDataTypePhysicalVariationList);
+
+ List<String> columns = new ArrayList<String>();
+ columns.add("col0");
+ ExprNodeColumnDesc col1Expr = new ExprNodeColumnDesc(typeInfo, "col0", "table", false);
+
+ List<ExprNodeDesc> children = new ArrayList<ExprNodeDesc>();
+ children.add(col1Expr);
+
+ //----------------------------------------------------------------------------------------------
+
+ String targetTypeName;
+ if (targetPrimitiveCategory == PrimitiveCategory.BYTE) {
+ targetTypeName = "tinyint";
+ } else if (targetPrimitiveCategory == PrimitiveCategory.SHORT) {
+ targetTypeName = "smallint";
+ } else if (targetPrimitiveCategory == PrimitiveCategory.LONG) {
+ targetTypeName = "bigint";
+ } else {
+ targetTypeName = targetPrimitiveCategory.name().toLowerCase();
+ }
+ targetTypeName = VectorRandomRowSource.getDecoratedTypeName(random, targetTypeName);
+ TypeInfo targetTypeInfo = TypeInfoUtils.getTypeInfoFromTypeString(targetTypeName);
+
+ //----------------------------------------------------------------------------------------------
+
+ String[] columnNames = columns.toArray(new String[0]);
+
+ Object[][] randomRows = rowSource.randomRows(100000);
+
+ VectorRandomBatchSource batchSource =
+ VectorRandomBatchSource.createInterestingBatches(
+ random,
+ rowSource,
+ randomRows,
+ null);
+
+ final int rowCount = randomRows.length;
+ Object[][] resultObjectsArray = new Object[CastStmtTestMode.count][];
+ for (int i = 0; i < CastStmtTestMode.count; i++) {
+
+ Object[] resultObjects = new Object[rowCount];
+ resultObjectsArray[i] = resultObjects;
+
+ CastStmtTestMode ifStmtTestMode = CastStmtTestMode.values()[i];
+ switch (ifStmtTestMode) {
+ case ROW_MODE:
+ if (!doRowCastTest(
+ typeInfo,
+ targetTypeInfo,
+ columns,
+ children,
+ randomRows,
+ rowSource.rowStructObjectInspector(),
+ resultObjects)) {
+ return;
+ }
+ break;
+ case ADAPTOR:
+ case VECTOR_EXPRESSION:
+ if (!doVectorCastTest(
+ typeInfo,
+ targetTypeInfo,
+ columns,
+ columnNames,
+ rowSource.typeInfos(),
+ rowSource.dataTypePhysicalVariations(),
+ children,
+ ifStmtTestMode,
+ batchSource,
+ resultObjects)) {
+ return;
+ }
+ break;
+ default:
+ throw new RuntimeException("Unexpected IF statement test mode " + ifStmtTestMode);
+ }
+ }
+
+ for (int i = 0; i < rowCount; i++) {
+ // Row-mode is the expected value.
+ Object expectedResult = resultObjectsArray[0][i];
+
+ for (int v = 1; v < CastStmtTestMode.count; v++) {
+ Object vectorResult = resultObjectsArray[v][i];
+ if (expectedResult == null || vectorResult == null) {
+ if (expectedResult != null || vectorResult != null) {
+ Assert.fail(
+ "Row " + i +
+ " sourceTypeName " + typeName +
+ " targetTypeName " + targetTypeName +
+ " " + CastStmtTestMode.values()[v] +
+ " result is NULL " + (vectorResult == null ? "YES" : "NO") +
+ " does not match row-mode expected result is NULL " +
+ (expectedResult == null ? "YES" : "NO"));
+ }
+ } else {
+
+ if (isDecimal64 && expectedResult instanceof LongWritable) {
+
+ HiveDecimalWritable expectedHiveDecimalWritable = new HiveDecimalWritable(0);
+ expectedHiveDecimalWritable.deserialize64(
+ ((LongWritable) expectedResult).get(), decimal64Scale);
+ expectedResult = expectedHiveDecimalWritable;
+ }
+
+ if (!expectedResult.equals(vectorResult)) {
+ Assert.fail(
+ "Row " + i +
+ " sourceTypeName " + typeName +
+ " targetTypeName " + targetTypeName +
+ " " + CastStmtTestMode.values()[v] +
+ " result " + vectorResult.toString() +
+ " (" + vectorResult.getClass().getSimpleName() + ")" +
+ " does not match row-mode expected result " + expectedResult.toString() +
+ " (" + expectedResult.getClass().getSimpleName() + ")");
+ }
+ }
+ }
+ }
+ }
+
+ private boolean doRowCastTest(TypeInfo typeInfo, TypeInfo targetTypeInfo,
+ List<String> columns, List<ExprNodeDesc> children,
+ Object[][] randomRows, ObjectInspector rowInspector, Object[] resultObjects)
+ throws Exception {
+
+ GenericUDF udf;
+ try {
+ udf = VectorizationContext.getGenericUDFForCast(targetTypeInfo);
+ } catch (HiveException e) {
+ return false;
+ }
+
+ ExprNodeGenericFuncDesc exprDesc =
+ new ExprNodeGenericFuncDesc(targetTypeInfo, udf, children);
+
+ /*
+ System.out.println(
+ "*DEBUG* typeInfo " + typeInfo.toString() +
+ " targetTypeInfo " + targetTypeInfo +
+ " castStmtTestMode ROW_MODE" +
+ " exprDesc " + exprDesc.toString());
+ */
+
+ HiveConf hiveConf = new HiveConf();
+ ExprNodeEvaluator evaluator =
+ ExprNodeEvaluatorFactory.get(exprDesc, hiveConf);
+ try {
+ evaluator.initialize(rowInspector);
+ } catch (HiveException e) {
+ return false;
+ }
+
+ ObjectInspector objectInspector = TypeInfoUtils
+ .getStandardWritableObjectInspectorFromTypeInfo(targetTypeInfo);
+
+ final int rowCount = randomRows.length;
+ for (int i = 0; i < rowCount; i++) {
+ Object[] row = randomRows[i];
+ Object result = evaluator.evaluate(row);
+ Object copyResult =
+ ObjectInspectorUtils.copyToStandardObject(
+ result, objectInspector, ObjectInspectorCopyOption.WRITABLE);
+ resultObjects[i] = copyResult;
+ }
+
+ return true;
+ }
+
+ private void extractResultObjects(VectorizedRowBatch batch, int rowIndex,
+ VectorExtractRow resultVectorExtractRow, Object[] scrqtchRow, Object[] resultObjects) {
+ // UNDONE: selectedInUse
+ for (int i = 0; i < batch.size; i++) {
+ resultVectorExtractRow.extractRow(batch, i, scrqtchRow);
+
+ // UNDONE: Need to copy the object.
+ resultObjects[rowIndex++] = scrqtchRow[0];
+ }
+ }
+
+ private boolean doVectorCastTest(TypeInfo typeInfo, TypeInfo targetTypeInfo,
+ List<String> columns, String[] columnNames,
+ TypeInfo[] typeInfos, DataTypePhysicalVariation[] dataTypePhysicalVariations,
+ List<ExprNodeDesc> children,
+ CastStmtTestMode castStmtTestMode,
+ VectorRandomBatchSource batchSource,
+ Object[] resultObjects)
+ throws Exception {
+
+ GenericUDF udf;
+ try {
+ udf = VectorizationContext.getGenericUDFForCast(targetTypeInfo);
+ } catch (HiveException e) {
+ return false;
+ }
+
+ ExprNodeGenericFuncDesc exprDesc =
+ new ExprNodeGenericFuncDesc(targetTypeInfo, udf, children);
+
+ HiveConf hiveConf = new HiveConf();
+ if (castStmtTestMode == CastStmtTestMode.ADAPTOR) {
+ hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_VECTOR_ADAPTOR_OVERRIDE, true);
+ }
+
+ VectorizationContext vectorizationContext =
+ new VectorizationContext(
+ "name",
+ columns,
+ Arrays.asList(typeInfos),
+ Arrays.asList(dataTypePhysicalVariations),
+ hiveConf);
+ VectorExpression vectorExpression = vectorizationContext.getVectorExpression(exprDesc);
+ vectorExpression.transientInit();
+
+ /*
+ System.out.println(
+ "*DEBUG* typeInfo " + typeInfo.toString() +
+ " targetTypeInfo " + targetTypeInfo +
+ " castStmtTestMode " + castStmtTestMode +
+ " vectorExpression " + vectorExpression.toString());
+ */
+
+ VectorRandomRowSource rowSource = batchSource.getRowSource();
+ VectorizedRowBatchCtx batchContext =
+ new VectorizedRowBatchCtx(
+ columnNames,
+ rowSource.typeInfos(),
+ rowSource.dataTypePhysicalVariations(),
+ /* dataColumnNums */ null,
+ /* partitionColumnCount */ 0,
+ /* virtualColumnCount */ 0,
+ /* neededVirtualColumns */ null,
+ vectorizationContext.getScratchColumnTypeNames(),
+ vectorizationContext.getScratchDataTypePhysicalVariations());
+
+ VectorizedRowBatch batch = batchContext.createVectorizedRowBatch();
+
+ VectorExtractRow resultVectorExtractRow = new VectorExtractRow();
+
+ resultVectorExtractRow.init(
+ new TypeInfo[] { targetTypeInfo }, new int[] { vectorExpression.getOutputColumnNum() });
+ Object[] scrqtchRow = new Object[1];
+
+ batchSource.resetBatchIteration();
+ int rowIndex = 0;
+ while (true) {
+ if (!batchSource.fillNextBatch(batch)) {
+ break;
+ }
+ vectorExpression.evaluate(batch);
+ extractResultObjects(batch, rowIndex, resultVectorExtractRow, scrqtchRow, resultObjects);
+ rowIndex += batch.size;
+ }
+
+ return true;
+ }
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/25aaf7db/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTypeCasts.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTypeCasts.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTypeCasts.java
index 8499da6..58ed151 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTypeCasts.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTypeCasts.java
@@ -33,6 +33,7 @@ import java.util.concurrent.TimeUnit;
import junit.framework.Assert;
+import org.apache.hadoop.hive.common.type.DataTypePhysicalVariation;
import org.apache.hadoop.hive.common.type.HiveDecimal;
import org.apache.hadoop.hive.common.type.RandomTypeUtil;
import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
@@ -167,6 +168,9 @@ public class TestVectorTypeCasts {
LongColumnVector resultV = (LongColumnVector) b.cols[1];
b.cols[0].noNulls = true;
VectorExpression expr = new CastTimestampToLong(0, 1);
+ expr.setOutputTypeInfo(TypeInfoFactory.longTypeInfo);
+ expr.setOutputDataTypePhysicalVariation(DataTypePhysicalVariation.NONE);
+ expr.transientInit();
expr.evaluate(b);
for (int i = 0; i < longValues.length; i++) {
long actual = resultV.vector[i];
http://git-wip-us.apache.org/repos/asf/hive/blob/25aaf7db/ql/src/test/results/clientpositive/llap/vector_decimal_aggregate.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vector_decimal_aggregate.q.out b/ql/src/test/results/clientpositive/llap/vector_decimal_aggregate.q.out
index 902d137..6cd1e8d 100644
--- a/ql/src/test/results/clientpositive/llap/vector_decimal_aggregate.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_decimal_aggregate.q.out
@@ -806,7 +806,7 @@ POSTHOOK: Input: default@decimal_vgby_small
626923679 1024 9723.40270 -9778.95135 10541.05247 10.29399655273437500000000000000 5742.091453325365 5744.897264122335 1024 11646 -11712 12641 12.3447 6877.306686989158 6880.6672084147185
6981 2 -515.62107 -515.62107 -1031.24214 -515.62107000000000000000000000000 0.0 0.0 3 6984454 -618 6983218 2327739.3333 3292794.518850853 4032833.1995089175
762 1 1531.21941 1531.21941 1531.21941 1531.21941000000000000000000000000 0.0 NULL 2 6984454 1834 6986288 3493144.0000 3491310.0 4937457.95244881
-NULL 3072 9318.43514 -4298.15135 5018444.11392 1633.60811000000000000000000000000 5695.4830839098695 5696.410309489299 3072 11161 -5148 6010880 1956.6667 6821.647911041892 6822.758476439734
+NULL 3072 9318.43514 -4298.15135 5018444.11392 NULL 5695.4830839098695 5696.410309489299 3072 11161 -5148 6010880 1956.6667 6821.647911041892 6822.758476439734
PREHOOK: query: SELECT SUM(HASH(*))
FROM (SELECT cint,
COUNT(cdecimal1), MAX(cdecimal1), MIN(cdecimal1), SUM(cdecimal1), AVG(cdecimal1), STDDEV_POP(cdecimal1), STDDEV_SAMP(cdecimal1),
@@ -825,4 +825,4 @@ FROM (SELECT cint,
POSTHOOK: type: QUERY
POSTHOOK: Input: default@decimal_vgby_small
#### A masked pattern was here ####
-96966670826
+96673467876
http://git-wip-us.apache.org/repos/asf/hive/blob/25aaf7db/ql/src/test/results/clientpositive/spark/timestamp_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/timestamp_1.q.out b/ql/src/test/results/clientpositive/spark/timestamp_1.q.out
index d3ca5cf..fab69ec 100644
--- a/ql/src/test/results/clientpositive/spark/timestamp_1.q.out
+++ b/ql/src/test/results/clientpositive/spark/timestamp_1.q.out
@@ -46,7 +46,7 @@ POSTHOOK: query: select cast(t as tinyint) from timestamp_1 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_1
#### A masked pattern was here ####
-77
+NULL
PREHOOK: query: select cast(t as smallint) from timestamp_1 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_1
@@ -55,7 +55,7 @@ POSTHOOK: query: select cast(t as smallint) from timestamp_1 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_1
#### A masked pattern was here ####
--4787
+NULL
PREHOOK: query: select cast(t as int) from timestamp_1 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_1
@@ -129,7 +129,7 @@ POSTHOOK: query: select cast(t as tinyint) from timestamp_1 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_1
#### A masked pattern was here ####
-77
+NULL
PREHOOK: query: select cast(t as smallint) from timestamp_1 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_1
@@ -138,7 +138,7 @@ POSTHOOK: query: select cast(t as smallint) from timestamp_1 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_1
#### A masked pattern was here ####
--4787
+NULL
PREHOOK: query: select cast(t as int) from timestamp_1 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_1
@@ -212,7 +212,7 @@ POSTHOOK: query: select cast(t as tinyint) from timestamp_1 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_1
#### A masked pattern was here ####
-77
+NULL
PREHOOK: query: select cast(t as smallint) from timestamp_1 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_1
@@ -221,7 +221,7 @@ POSTHOOK: query: select cast(t as smallint) from timestamp_1 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_1
#### A masked pattern was here ####
--4787
+NULL
PREHOOK: query: select cast(t as int) from timestamp_1 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_1
@@ -295,7 +295,7 @@ POSTHOOK: query: select cast(t as tinyint) from timestamp_1 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_1
#### A masked pattern was here ####
-77
+NULL
PREHOOK: query: select cast(t as smallint) from timestamp_1 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_1
@@ -304,7 +304,7 @@ POSTHOOK: query: select cast(t as smallint) from timestamp_1 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_1
#### A masked pattern was here ####
--4787
+NULL
PREHOOK: query: select cast(t as int) from timestamp_1 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_1
@@ -378,7 +378,7 @@ POSTHOOK: query: select cast(t as tinyint) from timestamp_1 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_1
#### A masked pattern was here ####
-77
+NULL
PREHOOK: query: select cast(t as smallint) from timestamp_1 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_1
@@ -387,7 +387,7 @@ POSTHOOK: query: select cast(t as smallint) from timestamp_1 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_1
#### A masked pattern was here ####
--4787
+NULL
PREHOOK: query: select cast(t as int) from timestamp_1 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_1
@@ -461,7 +461,7 @@ POSTHOOK: query: select cast(t as tinyint) from timestamp_1 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_1
#### A masked pattern was here ####
-77
+NULL
PREHOOK: query: select cast(t as smallint) from timestamp_1 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_1
@@ -470,7 +470,7 @@ POSTHOOK: query: select cast(t as smallint) from timestamp_1 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_1
#### A masked pattern was here ####
--4787
+NULL
PREHOOK: query: select cast(t as int) from timestamp_1 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_1
http://git-wip-us.apache.org/repos/asf/hive/blob/25aaf7db/ql/src/test/results/clientpositive/spark/timestamp_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/timestamp_2.q.out b/ql/src/test/results/clientpositive/spark/timestamp_2.q.out
index f9bfb09..9a05dfe 100644
--- a/ql/src/test/results/clientpositive/spark/timestamp_2.q.out
+++ b/ql/src/test/results/clientpositive/spark/timestamp_2.q.out
@@ -46,7 +46,7 @@ POSTHOOK: query: select cast(t as tinyint) from timestamp_2 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_2
#### A masked pattern was here ####
-77
+NULL
PREHOOK: query: select cast(t as smallint) from timestamp_2 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_2
@@ -55,7 +55,7 @@ POSTHOOK: query: select cast(t as smallint) from timestamp_2 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_2
#### A masked pattern was here ####
--4787
+NULL
PREHOOK: query: select cast(t as int) from timestamp_2 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_2
@@ -129,7 +129,7 @@ POSTHOOK: query: select cast(t as tinyint) from timestamp_2 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_2
#### A masked pattern was here ####
-77
+NULL
PREHOOK: query: select cast(t as smallint) from timestamp_2 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_2
@@ -138,7 +138,7 @@ POSTHOOK: query: select cast(t as smallint) from timestamp_2 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_2
#### A masked pattern was here ####
--4787
+NULL
PREHOOK: query: select cast(t as int) from timestamp_2 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_2
@@ -212,7 +212,7 @@ POSTHOOK: query: select cast(t as tinyint) from timestamp_2 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_2
#### A masked pattern was here ####
-77
+NULL
PREHOOK: query: select cast(t as smallint) from timestamp_2 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_2
@@ -221,7 +221,7 @@ POSTHOOK: query: select cast(t as smallint) from timestamp_2 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_2
#### A masked pattern was here ####
--4787
+NULL
PREHOOK: query: select cast(t as int) from timestamp_2 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_2
@@ -295,7 +295,7 @@ POSTHOOK: query: select cast(t as tinyint) from timestamp_2 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_2
#### A masked pattern was here ####
-77
+NULL
PREHOOK: query: select cast(t as smallint) from timestamp_2 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_2
@@ -304,7 +304,7 @@ POSTHOOK: query: select cast(t as smallint) from timestamp_2 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_2
#### A masked pattern was here ####
--4787
+NULL
PREHOOK: query: select cast(t as int) from timestamp_2 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_2
@@ -378,7 +378,7 @@ POSTHOOK: query: select cast(t as tinyint) from timestamp_2 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_2
#### A masked pattern was here ####
-77
+NULL
PREHOOK: query: select cast(t as smallint) from timestamp_2 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_2
@@ -387,7 +387,7 @@ POSTHOOK: query: select cast(t as smallint) from timestamp_2 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_2
#### A masked pattern was here ####
--4787
+NULL
PREHOOK: query: select cast(t as int) from timestamp_2 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_2
@@ -461,7 +461,7 @@ POSTHOOK: query: select cast(t as tinyint) from timestamp_2 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_2
#### A masked pattern was here ####
-77
+NULL
PREHOOK: query: select cast(t as smallint) from timestamp_2 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_2
@@ -470,7 +470,7 @@ POSTHOOK: query: select cast(t as smallint) from timestamp_2 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_2
#### A masked pattern was here ####
--4787
+NULL
PREHOOK: query: select cast(t as int) from timestamp_2 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_2
http://git-wip-us.apache.org/repos/asf/hive/blob/25aaf7db/ql/src/test/results/clientpositive/spark/timestamp_3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/timestamp_3.q.out b/ql/src/test/results/clientpositive/spark/timestamp_3.q.out
index 0664abf..6d59269 100644
--- a/ql/src/test/results/clientpositive/spark/timestamp_3.q.out
+++ b/ql/src/test/results/clientpositive/spark/timestamp_3.q.out
@@ -46,7 +46,7 @@ POSTHOOK: query: select cast(t as tinyint) from timestamp_3 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_3
#### A masked pattern was here ####
-48
+NULL
PREHOOK: query: select cast(t as smallint) from timestamp_3 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_3
@@ -55,7 +55,7 @@ POSTHOOK: query: select cast(t as smallint) from timestamp_3 limit 1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@timestamp_3
#### A masked pattern was here ####
--31184
+NULL
PREHOOK: query: select cast(t as int) from timestamp_3 limit 1
PREHOOK: type: QUERY
PREHOOK: Input: default@timestamp_3