You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ab...@apache.org on 2020/10/01 06:19:57 UTC
[hive] branch master updated: HIVE-23852: add support for Date type
in ReduceSink operator for native vector sink (#1274) (Panagiotis
Garefalakis reviewed by Mustafa Iman, Laszlo Bodor)
This is an automated email from the ASF dual-hosted git repository.
abstractdog pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git
The following commit(s) were added to refs/heads/master by this push:
new 7cd4553 HIVE-23852: add support for Date type in ReduceSink operator for native vector sink (#1274) (Panagiotis Garefalakis reviewed by Mustafa Iman, Laszlo Bodor)
7cd4553 is described below
commit 7cd455344e6fd9dce383feb72a2dbe0610523152
Author: Panagiotis Garefalakis <pg...@cloudera.com>
AuthorDate: Thu Oct 1 07:19:45 2020 +0100
HIVE-23852: add support for Date type in ReduceSink operator for native vector sink (#1274) (Panagiotis Garefalakis reviewed by Mustafa Iman, Laszlo Bodor)
Change-Id: I0b151b72d70f3f57278144def5b64a063cd77623
TS table
Change-Id: I8df436c4d493e1647f187ae9928fefa7d64a74d9
Polishing patch
Change-Id: I571047ab42bcf0091f2766339d6b70048cb73a06
Cleaning
Change-Id: I42e720365a659e90074d9c94debbb9b083732f71
Proper logging
Change-Id: I872883d13cd412d1cd0b5193f4225e0017f62816
---
.../keyseries/VectorKeySeriesLongSerialized.java | 4 +-
.../fast/VectorMapJoinFastLongHashUtil.java | 5 +-
.../fast/VectorMapJoinFastTableContainer.java | 1 +
.../VectorMapJoinOptimizedLongCommon.java | 3 +
.../hive/ql/optimizer/physical/Vectorizer.java | 8 +-
.../hadoop/hive/ql/plan/VectorMapJoinDesc.java | 11 +-
.../clientpositive/probedecode_static_keytype.q | 52 ++
.../llap/probedecode_static_keytype.q.out | 564 +++++++++++++++++++++
.../llap/vector_partitioned_date_time.q.out | 8 +-
9 files changed, 645 insertions(+), 11 deletions(-)
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/keyseries/VectorKeySeriesLongSerialized.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/keyseries/VectorKeySeriesLongSerialized.java
index d40421d..33918db 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/keyseries/VectorKeySeriesLongSerialized.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/keyseries/VectorKeySeriesLongSerialized.java
@@ -222,7 +222,6 @@ public class VectorKeySeriesLongSerialized<T extends SerializeWrite>
private void serialize(int pos, long value) throws IOException {
serializeWrite.setAppend(output);
- // UNDONE: Add support for DATE, TIMESTAMP, INTERVAL_YEAR_MONTH, INTERVAL_DAY_TIME...
switch (primitiveCategory) {
case BOOLEAN:
serializeWrite.writeBoolean(value != 0);
@@ -236,6 +235,9 @@ public class VectorKeySeriesLongSerialized<T extends SerializeWrite>
case INT:
serializeWrite.writeInt((int) value);
break;
+ case DATE:
+ serializeWrite.writeDate((int) value);
+ break;
case LONG:
serializeWrite.writeLong(value);
break;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastLongHashUtil.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastLongHashUtil.java
index 65393e6..d3bda21 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastLongHashUtil.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastLongHashUtil.java
@@ -26,7 +26,7 @@ import org.apache.hadoop.hive.serde2.binarysortable.fast.BinarySortableDeseriali
public class VectorMapJoinFastLongHashUtil {
public static long deserializeLongKey(BinarySortableDeserializeRead keyBinarySortableDeserializeRead,
- HashTableKeyType hashTableKeyType) throws IOException {
+ HashTableKeyType hashTableKeyType) throws RuntimeException {
long key = 0;
switch (hashTableKeyType) {
case BOOLEAN:
@@ -41,6 +41,9 @@ public class VectorMapJoinFastLongHashUtil {
case INT:
key = (long) keyBinarySortableDeserializeRead.currentInt;
break;
+ case DATE:
+ key = (long) keyBinarySortableDeserializeRead.currentDateWritable.getDays();
+ break;
case LONG:
key = keyBinarySortableDeserializeRead.currentLong;
break;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastTableContainer.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastTableContainer.java
index c475181..ffc9fbf 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastTableContainer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastTableContainer.java
@@ -117,6 +117,7 @@ public class VectorMapJoinFastTableContainer implements VectorMapJoinTableContai
case BYTE:
case SHORT:
case INT:
+ case DATE:
case LONG:
switch (hashTableKind) {
case HASH_MAP:
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/optimized/VectorMapJoinOptimizedLongCommon.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/optimized/VectorMapJoinOptimizedLongCommon.java
index 16f8b02..c402fea 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/optimized/VectorMapJoinOptimizedLongCommon.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/optimized/VectorMapJoinOptimizedLongCommon.java
@@ -82,6 +82,9 @@ public class VectorMapJoinOptimizedLongCommon {
case INT:
keyBinarySortableSerializeWrite.writeInt((int) key);
break;
+ case DATE:
+ keyBinarySortableSerializeWrite.writeDate((int) key);
+ break;
case LONG:
keyBinarySortableSerializeWrite.writeLong(key);
break;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
index 99b272c..114d684 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
@@ -3382,6 +3382,9 @@ public class Vectorizer implements PhysicalPlanResolver {
case INT:
hashTableKeyType = HashTableKeyType.INT;
break;
+ case DATE:
+ hashTableKeyType = HashTableKeyType.DATE;
+ break;
case LONG:
hashTableKeyType = HashTableKeyType.LONG;
break;
@@ -3434,6 +3437,7 @@ public class Vectorizer implements PhysicalPlanResolver {
case BYTE:
case SHORT:
case INT:
+ case DATE:
case LONG:
switch (vectorMapJoinVariation) {
case INNER:
@@ -3989,11 +3993,13 @@ public class Vectorizer implements PhysicalPlanResolver {
case BYTE:
case SHORT:
case INT:
+ case DATE:
case LONG:
reduceSinkKeyType = VectorReduceSinkDesc.ReduceSinkKeyType.LONG;
break;
default:
- // Other integer types not supported yet.
+ // For any remaining Long CV types use default multi-key
+ LOG.warn("Unsupported Long-CV key type {} defaulted to multi-key ReduceSink", primitiveCategory);
break;
}
}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorMapJoinDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorMapJoinDesc.java
index f32a230..e68e3bd 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorMapJoinDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorMapJoinDesc.java
@@ -57,24 +57,27 @@ public class VectorMapJoinDesc extends AbstractVectorDesc {
BYTE,
SHORT,
INT,
+ DATE,
LONG,
STRING,
MULTI_KEY;
public PrimitiveTypeInfo getPrimitiveTypeInfo() {
switch (this) {
+ case NONE:
+ return TypeInfoFactory.voidTypeInfo;
case BOOLEAN:
return TypeInfoFactory.booleanTypeInfo;
case BYTE:
return TypeInfoFactory.byteTypeInfo;
+ case SHORT:
+ return TypeInfoFactory.shortTypeInfo;
+ case DATE:
+ return TypeInfoFactory.dateTypeInfo;
case INT:
return TypeInfoFactory.intTypeInfo;
case LONG:
return TypeInfoFactory.longTypeInfo;
- case NONE:
- return TypeInfoFactory.voidTypeInfo;
- case SHORT:
- return TypeInfoFactory.shortTypeInfo;
case STRING:
return TypeInfoFactory.stringTypeInfo;
case MULTI_KEY:
diff --git a/ql/src/test/queries/clientpositive/probedecode_static_keytype.q b/ql/src/test/queries/clientpositive/probedecode_static_keytype.q
new file mode 100644
index 0000000..960e6e9
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/probedecode_static_keytype.q
@@ -0,0 +1,52 @@
+set hive.stats.column.autogather=false;
+set hive.mapred.mode=nonstrict;
+set hive.explain.user=false;
+SET hive.auto.convert.join=true;
+SET hive.auto.convert.join.noconditionaltask=true;
+SET hive.auto.convert.join.noconditionaltask.size=1000000000;
+SET hive.vectorized.execution.enabled=true;
+set hive.vectorized.execution.mapjoin.native.fast.hashtable.enabled=true;
+set hive.fetch.task.conversion=none;
+SET mapred.min.split.size=1000;
+SET mapred.max.split.size=5000;
+
+SET hive.optimize.scan.probedecode=true;
+
+
+CREATE TABLE item_dim_dt (key1 DATE, name string) stored as ORC;
+CREATE TABLE orders_fact_dt (nokey int, key2 DATE, dt timestamp) stored as ORC;
+
+INSERT INTO item_dim_dt values('2001-01-30', "Item 101");
+INSERT INTO item_dim_dt values('2002-01-30', "Item 102");
+
+INSERT INTO orders_fact_dt values(12345, '2001-01-30', '2011-01-30 00:00:00');
+INSERT INTO orders_fact_dt values(23456, '2004-01-30', '2014-02-30 00:00:00');
+INSERT INTO orders_fact_dt values(34567, '2008-01-30', '2018-03-30 00:00:00');
+INSERT INTO orders_fact_dt values(45678, '2002-01-30', '2012-04-30 00:00:00');
+INSERT INTO orders_fact_dt values(56789, '2009-01-30', '2019-05-30 00:00:00');
+INSERT INTO orders_fact_dt values(67891, '2010-01-30', '2020-06-30 00:00:00');
+
+-- Reduce Sink Vectorization -> Expected className: VectorReduceSinkLongOperator
+EXPLAIN VECTORIZATION DETAIL select key1, key2, name, dt from orders_fact_dt join item_dim_dt on (orders_fact_dt.key2 = item_dim_dt.key1);
+-- two keys match, the remaining rows can be skipped
+select key1, key2, name, dt from orders_fact_dt join item_dim_dt on (orders_fact_dt.key2 = item_dim_dt.key1);
+
+
+CREATE TABLE item_dim_ts (key1 timestamp, name string) stored as ORC;
+CREATE TABLE orders_fact_ts (nokey int, key2 timestamp, dt timestamp) stored as ORC;
+
+INSERT INTO item_dim_ts values('2001-01-30 00:00:00', "Item 101");
+INSERT INTO item_dim_ts values('2002-01-30 00:00:00', "Item 102");
+
+INSERT INTO orders_fact_ts values(12345, '2001-01-30 00:00:00', '2011-01-30 00:00:00');
+INSERT INTO orders_fact_ts values(23456, '2004-01-30 00:00:00', '2014-02-30 00:00:00');
+INSERT INTO orders_fact_ts values(34567, '2008-01-30 00:00:00', '2018-03-30 00:00:00');
+INSERT INTO orders_fact_ts values(45678, '2002-01-30 00:00:00', '2012-04-30 00:00:00');
+INSERT INTO orders_fact_ts values(56789, '2009-01-30 00:00:00', '2019-05-30 00:00:00');
+INSERT INTO orders_fact_ts values(67891, '2010-01-30 00:00:00', '2020-06-30 00:00:00');
+
+-- Reduce Sink Vectorization -> Expected className: VectorReduceSinkMultiKeyOperator
+EXPLAIN VECTORIZATION DETAIL select key1, key2, name, dt from orders_fact_ts join item_dim_ts on (orders_fact_ts.key2 = item_dim_ts.key1);
+
+-- two keys match, the remaining rows can be skipped
+select key1, key2, name, dt from orders_fact_ts join item_dim_ts on (orders_fact_ts.key2 = item_dim_ts.key1);
diff --git a/ql/src/test/results/clientpositive/llap/probedecode_static_keytype.q.out b/ql/src/test/results/clientpositive/llap/probedecode_static_keytype.q.out
new file mode 100644
index 0000000..4821a41
--- /dev/null
+++ b/ql/src/test/results/clientpositive/llap/probedecode_static_keytype.q.out
@@ -0,0 +1,564 @@
+PREHOOK: query: CREATE TABLE item_dim_dt (key1 DATE, name string) stored as ORC
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@item_dim_dt
+POSTHOOK: query: CREATE TABLE item_dim_dt (key1 DATE, name string) stored as ORC
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@item_dim_dt
+PREHOOK: query: CREATE TABLE orders_fact_dt (nokey int, key2 DATE, dt timestamp) stored as ORC
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@orders_fact_dt
+POSTHOOK: query: CREATE TABLE orders_fact_dt (nokey int, key2 DATE, dt timestamp) stored as ORC
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@orders_fact_dt
+PREHOOK: query: INSERT INTO item_dim_dt values('2001-01-30', "Item 101")
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@item_dim_dt
+POSTHOOK: query: INSERT INTO item_dim_dt values('2001-01-30', "Item 101")
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@item_dim_dt
+POSTHOOK: Lineage: item_dim_dt.key1 SCRIPT []
+POSTHOOK: Lineage: item_dim_dt.name SCRIPT []
+PREHOOK: query: INSERT INTO item_dim_dt values('2002-01-30', "Item 102")
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@item_dim_dt
+POSTHOOK: query: INSERT INTO item_dim_dt values('2002-01-30', "Item 102")
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@item_dim_dt
+POSTHOOK: Lineage: item_dim_dt.key1 SCRIPT []
+POSTHOOK: Lineage: item_dim_dt.name SCRIPT []
+PREHOOK: query: INSERT INTO orders_fact_dt values(12345, '2001-01-30', '2011-01-30 00:00:00')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@orders_fact_dt
+POSTHOOK: query: INSERT INTO orders_fact_dt values(12345, '2001-01-30', '2011-01-30 00:00:00')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@orders_fact_dt
+POSTHOOK: Lineage: orders_fact_dt.dt SCRIPT []
+POSTHOOK: Lineage: orders_fact_dt.key2 SCRIPT []
+POSTHOOK: Lineage: orders_fact_dt.nokey SCRIPT []
+PREHOOK: query: INSERT INTO orders_fact_dt values(23456, '2004-01-30', '2014-02-30 00:00:00')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@orders_fact_dt
+POSTHOOK: query: INSERT INTO orders_fact_dt values(23456, '2004-01-30', '2014-02-30 00:00:00')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@orders_fact_dt
+POSTHOOK: Lineage: orders_fact_dt.dt SCRIPT []
+POSTHOOK: Lineage: orders_fact_dt.key2 SCRIPT []
+POSTHOOK: Lineage: orders_fact_dt.nokey SCRIPT []
+PREHOOK: query: INSERT INTO orders_fact_dt values(34567, '2008-01-30', '2018-03-30 00:00:00')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@orders_fact_dt
+POSTHOOK: query: INSERT INTO orders_fact_dt values(34567, '2008-01-30', '2018-03-30 00:00:00')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@orders_fact_dt
+POSTHOOK: Lineage: orders_fact_dt.dt SCRIPT []
+POSTHOOK: Lineage: orders_fact_dt.key2 SCRIPT []
+POSTHOOK: Lineage: orders_fact_dt.nokey SCRIPT []
+PREHOOK: query: INSERT INTO orders_fact_dt values(45678, '2002-01-30', '2012-04-30 00:00:00')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@orders_fact_dt
+POSTHOOK: query: INSERT INTO orders_fact_dt values(45678, '2002-01-30', '2012-04-30 00:00:00')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@orders_fact_dt
+POSTHOOK: Lineage: orders_fact_dt.dt SCRIPT []
+POSTHOOK: Lineage: orders_fact_dt.key2 SCRIPT []
+POSTHOOK: Lineage: orders_fact_dt.nokey SCRIPT []
+PREHOOK: query: INSERT INTO orders_fact_dt values(56789, '2009-01-30', '2019-05-30 00:00:00')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@orders_fact_dt
+POSTHOOK: query: INSERT INTO orders_fact_dt values(56789, '2009-01-30', '2019-05-30 00:00:00')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@orders_fact_dt
+POSTHOOK: Lineage: orders_fact_dt.dt SCRIPT []
+POSTHOOK: Lineage: orders_fact_dt.key2 SCRIPT []
+POSTHOOK: Lineage: orders_fact_dt.nokey SCRIPT []
+PREHOOK: query: INSERT INTO orders_fact_dt values(67891, '2010-01-30', '2020-06-30 00:00:00')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@orders_fact_dt
+POSTHOOK: query: INSERT INTO orders_fact_dt values(67891, '2010-01-30', '2020-06-30 00:00:00')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@orders_fact_dt
+POSTHOOK: Lineage: orders_fact_dt.dt SCRIPT []
+POSTHOOK: Lineage: orders_fact_dt.key2 SCRIPT []
+POSTHOOK: Lineage: orders_fact_dt.nokey SCRIPT []
+PREHOOK: query: EXPLAIN VECTORIZATION DETAIL select key1, key2, name, dt from orders_fact_dt join item_dim_dt on (orders_fact_dt.key2 = item_dim_dt.key1)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@item_dim_dt
+PREHOOK: Input: default@orders_fact_dt
+#### A masked pattern was here ####
+POSTHOOK: query: EXPLAIN VECTORIZATION DETAIL select key1, key2, name, dt from orders_fact_dt join item_dim_dt on (orders_fact_dt.key2 = item_dim_dt.key1)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@item_dim_dt
+POSTHOOK: Input: default@orders_fact_dt
+#### A masked pattern was here ####
+PLAN VECTORIZATION:
+ enabled: true
+ enabledConditionsMet: [hive.vectorized.execution.enabled IS true]
+
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-1
+ Tez
+#### A masked pattern was here ####
+ Edges:
+ Map 1 <- Map 2 (BROADCAST_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: orders_fact_dt
+ filterExpr: key2 is not null (type: boolean)
+ probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_25_container, bigKeyColName:key2, smallTablePos:1, keyRatio:1.0
+ Statistics: Num rows: 6 Data size: 576 Basic stats: COMPLETE Column stats: NONE
+ TableScan Vectorization:
+ native: true
+ vectorizationSchemaColumns: [0:nokey:int, 1:key2:date, 2:dt:timestamp, 3:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
+ Filter Operator
+ Filter Vectorization:
+ className: VectorFilterOperator
+ native: true
+ predicateExpression: SelectColumnIsNotNull(col 1:date)
+ predicate: key2 is not null (type: boolean)
+ Statistics: Num rows: 6 Data size: 576 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: key2 (type: date), dt (type: timestamp)
+ outputColumnNames: _col0, _col1
+ Select Vectorization:
+ className: VectorSelectOperator
+ native: true
+ projectedOutputColumnNums: [1, 2]
+ Statistics: Num rows: 6 Data size: 576 Basic stats: COMPLETE Column stats: NONE
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ keys:
+ 0 _col0 (type: date)
+ 1 _col0 (type: date)
+ Map Join Vectorization:
+ bigTableKeyColumns: 1:date
+ bigTableRetainColumnNums: [1, 2]
+ bigTableValueColumns: 1:date, 2:timestamp
+ className: VectorMapJoinInnerLongOperator
+ native: true
+ nativeConditionsMet: hive.mapjoin.optimized.hashtable IS true, hive.vectorized.execution.mapjoin.native.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, One MapJoin Condition IS true, No nullsafe IS true, Small table vectorizes IS true, Fast Hash Table and No Hybrid Hash Join IS true
+ nonOuterSmallTableKeyMapping: []
+ projectedOutput: 1:date, 2:timestamp, 1:date, 4:string
+ smallTableValueMapping: 4:string
+ hashTableImplementationType: FAST
+ outputColumnNames: _col0, _col1, _col2, _col3
+ input vertices:
+ 1 Map 2
+ Statistics: Num rows: 6 Data size: 633 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col2 (type: date), _col0 (type: date), _col3 (type: string), _col1 (type: timestamp)
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Select Vectorization:
+ className: VectorSelectOperator
+ native: true
+ projectedOutputColumnNums: [1, 1, 4, 2]
+ Statistics: Num rows: 6 Data size: 633 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ File Sink Vectorization:
+ className: VectorFileSinkOperator
+ native: false
+ Statistics: Num rows: 6 Data size: 633 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ Execution mode: vectorized, llap
+ LLAP IO: all inputs
+ Map Vectorization:
+ enabled: true
+ enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true
+ inputFormatFeatureSupport: [DECIMAL_64]
+ featureSupportInUse: [DECIMAL_64]
+ inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ allNative: false
+ usesVectorUDFAdaptor: false
+ vectorized: true
+ rowBatchContext:
+ dataColumnCount: 3
+ includeColumns: [1, 2]
+ dataColumns: nokey:int, key2:date, dt:timestamp
+ partitionColumnCount: 0
+ scratchColumnTypeNames: [string]
+ Map 2
+ Map Operator Tree:
+ TableScan
+ alias: item_dim_dt
+ filterExpr: key1 is not null (type: boolean)
+ Statistics: Num rows: 2 Data size: 480 Basic stats: COMPLETE Column stats: NONE
+ TableScan Vectorization:
+ native: true
+ vectorizationSchemaColumns: [0:key1:date, 1:name:string, 2:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
+ Filter Operator
+ Filter Vectorization:
+ className: VectorFilterOperator
+ native: true
+ predicateExpression: SelectColumnIsNotNull(col 0:date)
+ predicate: key1 is not null (type: boolean)
+ Statistics: Num rows: 2 Data size: 480 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: key1 (type: date), name (type: string)
+ outputColumnNames: _col0, _col1
+ Select Vectorization:
+ className: VectorSelectOperator
+ native: true
+ projectedOutputColumnNums: [0, 1]
+ Statistics: Num rows: 2 Data size: 480 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: date)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: date)
+ Reduce Sink Vectorization:
+ className: VectorReduceSinkLongOperator
+ keyColumns: 0:date
+ native: true
+ nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez 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
+ valueColumns: 1:string
+ Statistics: Num rows: 2 Data size: 480 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col1 (type: string)
+ Execution mode: vectorized, llap
+ LLAP IO: all inputs
+ Map Vectorization:
+ enabled: true
+ enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true
+ inputFormatFeatureSupport: [DECIMAL_64]
+ featureSupportInUse: [DECIMAL_64]
+ inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ allNative: true
+ usesVectorUDFAdaptor: false
+ vectorized: true
+ rowBatchContext:
+ dataColumnCount: 2
+ includeColumns: [0, 1]
+ dataColumns: key1:date, name:string
+ partitionColumnCount: 0
+ scratchColumnTypeNames: []
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: select key1, key2, name, dt from orders_fact_dt join item_dim_dt on (orders_fact_dt.key2 = item_dim_dt.key1)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@item_dim_dt
+PREHOOK: Input: default@orders_fact_dt
+#### A masked pattern was here ####
+POSTHOOK: query: select key1, key2, name, dt from orders_fact_dt join item_dim_dt on (orders_fact_dt.key2 = item_dim_dt.key1)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@item_dim_dt
+POSTHOOK: Input: default@orders_fact_dt
+#### A masked pattern was here ####
+2002-01-30 2002-01-30 Item 102 2012-04-30 00:00:00
+2001-01-30 2001-01-30 Item 101 2011-01-30 00:00:00
+PREHOOK: query: CREATE TABLE item_dim_ts (key1 timestamp, name string) stored as ORC
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@item_dim_ts
+POSTHOOK: query: CREATE TABLE item_dim_ts (key1 timestamp, name string) stored as ORC
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@item_dim_ts
+PREHOOK: query: CREATE TABLE orders_fact_ts (nokey int, key2 timestamp, dt timestamp) stored as ORC
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@orders_fact_ts
+POSTHOOK: query: CREATE TABLE orders_fact_ts (nokey int, key2 timestamp, dt timestamp) stored as ORC
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@orders_fact_ts
+PREHOOK: query: INSERT INTO item_dim_ts values('2001-01-30 00:00:00', "Item 101")
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@item_dim_ts
+POSTHOOK: query: INSERT INTO item_dim_ts values('2001-01-30 00:00:00', "Item 101")
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@item_dim_ts
+POSTHOOK: Lineage: item_dim_ts.key1 SCRIPT []
+POSTHOOK: Lineage: item_dim_ts.name SCRIPT []
+PREHOOK: query: INSERT INTO item_dim_ts values('2002-01-30 00:00:00', "Item 102")
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@item_dim_ts
+POSTHOOK: query: INSERT INTO item_dim_ts values('2002-01-30 00:00:00', "Item 102")
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@item_dim_ts
+POSTHOOK: Lineage: item_dim_ts.key1 SCRIPT []
+POSTHOOK: Lineage: item_dim_ts.name SCRIPT []
+PREHOOK: query: INSERT INTO orders_fact_ts values(12345, '2001-01-30 00:00:00', '2011-01-30 00:00:00')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@orders_fact_ts
+POSTHOOK: query: INSERT INTO orders_fact_ts values(12345, '2001-01-30 00:00:00', '2011-01-30 00:00:00')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@orders_fact_ts
+POSTHOOK: Lineage: orders_fact_ts.dt SCRIPT []
+POSTHOOK: Lineage: orders_fact_ts.key2 SCRIPT []
+POSTHOOK: Lineage: orders_fact_ts.nokey SCRIPT []
+PREHOOK: query: INSERT INTO orders_fact_ts values(23456, '2004-01-30 00:00:00', '2014-02-30 00:00:00')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@orders_fact_ts
+POSTHOOK: query: INSERT INTO orders_fact_ts values(23456, '2004-01-30 00:00:00', '2014-02-30 00:00:00')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@orders_fact_ts
+POSTHOOK: Lineage: orders_fact_ts.dt SCRIPT []
+POSTHOOK: Lineage: orders_fact_ts.key2 SCRIPT []
+POSTHOOK: Lineage: orders_fact_ts.nokey SCRIPT []
+PREHOOK: query: INSERT INTO orders_fact_ts values(34567, '2008-01-30 00:00:00', '2018-03-30 00:00:00')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@orders_fact_ts
+POSTHOOK: query: INSERT INTO orders_fact_ts values(34567, '2008-01-30 00:00:00', '2018-03-30 00:00:00')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@orders_fact_ts
+POSTHOOK: Lineage: orders_fact_ts.dt SCRIPT []
+POSTHOOK: Lineage: orders_fact_ts.key2 SCRIPT []
+POSTHOOK: Lineage: orders_fact_ts.nokey SCRIPT []
+PREHOOK: query: INSERT INTO orders_fact_ts values(45678, '2002-01-30 00:00:00', '2012-04-30 00:00:00')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@orders_fact_ts
+POSTHOOK: query: INSERT INTO orders_fact_ts values(45678, '2002-01-30 00:00:00', '2012-04-30 00:00:00')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@orders_fact_ts
+POSTHOOK: Lineage: orders_fact_ts.dt SCRIPT []
+POSTHOOK: Lineage: orders_fact_ts.key2 SCRIPT []
+POSTHOOK: Lineage: orders_fact_ts.nokey SCRIPT []
+PREHOOK: query: INSERT INTO orders_fact_ts values(56789, '2009-01-30 00:00:00', '2019-05-30 00:00:00')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@orders_fact_ts
+POSTHOOK: query: INSERT INTO orders_fact_ts values(56789, '2009-01-30 00:00:00', '2019-05-30 00:00:00')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@orders_fact_ts
+POSTHOOK: Lineage: orders_fact_ts.dt SCRIPT []
+POSTHOOK: Lineage: orders_fact_ts.key2 SCRIPT []
+POSTHOOK: Lineage: orders_fact_ts.nokey SCRIPT []
+PREHOOK: query: INSERT INTO orders_fact_ts values(67891, '2010-01-30 00:00:00', '2020-06-30 00:00:00')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@orders_fact_ts
+POSTHOOK: query: INSERT INTO orders_fact_ts values(67891, '2010-01-30 00:00:00', '2020-06-30 00:00:00')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@orders_fact_ts
+POSTHOOK: Lineage: orders_fact_ts.dt SCRIPT []
+POSTHOOK: Lineage: orders_fact_ts.key2 SCRIPT []
+POSTHOOK: Lineage: orders_fact_ts.nokey SCRIPT []
+PREHOOK: query: EXPLAIN VECTORIZATION DETAIL select key1, key2, name, dt from orders_fact_ts join item_dim_ts on (orders_fact_ts.key2 = item_dim_ts.key1)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@item_dim_ts
+PREHOOK: Input: default@orders_fact_ts
+#### A masked pattern was here ####
+POSTHOOK: query: EXPLAIN VECTORIZATION DETAIL select key1, key2, name, dt from orders_fact_ts join item_dim_ts on (orders_fact_ts.key2 = item_dim_ts.key1)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@item_dim_ts
+POSTHOOK: Input: default@orders_fact_ts
+#### A masked pattern was here ####
+PLAN VECTORIZATION:
+ enabled: true
+ enabledConditionsMet: [hive.vectorized.execution.enabled IS true]
+
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-1
+ Tez
+#### A masked pattern was here ####
+ Edges:
+ Map 1 <- Map 2 (BROADCAST_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: orders_fact_ts
+ filterExpr: key2 is not null (type: boolean)
+ probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_25_container, bigKeyColName:key2, smallTablePos:1, keyRatio:1.0
+ Statistics: Num rows: 6 Data size: 480 Basic stats: COMPLETE Column stats: NONE
+ TableScan Vectorization:
+ native: true
+ vectorizationSchemaColumns: [0:nokey:int, 1:key2:timestamp, 2:dt:timestamp, 3:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
+ Filter Operator
+ Filter Vectorization:
+ className: VectorFilterOperator
+ native: true
+ predicateExpression: SelectColumnIsNotNull(col 1:timestamp)
+ predicate: key2 is not null (type: boolean)
+ Statistics: Num rows: 6 Data size: 480 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: key2 (type: timestamp), dt (type: timestamp)
+ outputColumnNames: _col0, _col1
+ Select Vectorization:
+ className: VectorSelectOperator
+ native: true
+ projectedOutputColumnNums: [1, 2]
+ Statistics: Num rows: 6 Data size: 480 Basic stats: COMPLETE Column stats: NONE
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ keys:
+ 0 _col0 (type: timestamp)
+ 1 _col0 (type: timestamp)
+ Map Join Vectorization:
+ bigTableKeyColumns: 1:timestamp
+ bigTableRetainColumnNums: [1, 2]
+ bigTableValueColumns: 1:timestamp, 2:timestamp
+ className: VectorMapJoinInnerMultiKeyOperator
+ native: true
+ nativeConditionsMet: hive.mapjoin.optimized.hashtable IS true, hive.vectorized.execution.mapjoin.native.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, One MapJoin Condition IS true, No nullsafe IS true, Small table vectorizes IS true, Fast Hash Table and No Hybrid Hash Join IS true
+ nonOuterSmallTableKeyMapping: []
+ projectedOutput: 1:timestamp, 2:timestamp, 1:timestamp, 4:string
+ smallTableValueMapping: 4:string
+ hashTableImplementationType: FAST
+ outputColumnNames: _col0, _col1, _col2, _col3
+ input vertices:
+ 1 Map 2
+ Statistics: Num rows: 6 Data size: 528 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col2 (type: timestamp), _col0 (type: timestamp), _col3 (type: string), _col1 (type: timestamp)
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Select Vectorization:
+ className: VectorSelectOperator
+ native: true
+ projectedOutputColumnNums: [1, 1, 4, 2]
+ Statistics: Num rows: 6 Data size: 528 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ File Sink Vectorization:
+ className: VectorFileSinkOperator
+ native: false
+ Statistics: Num rows: 6 Data size: 528 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ Execution mode: vectorized, llap
+ LLAP IO: all inputs
+ Map Vectorization:
+ enabled: true
+ enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true
+ inputFormatFeatureSupport: [DECIMAL_64]
+ featureSupportInUse: [DECIMAL_64]
+ inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ allNative: false
+ usesVectorUDFAdaptor: false
+ vectorized: true
+ rowBatchContext:
+ dataColumnCount: 3
+ includeColumns: [1, 2]
+ dataColumns: nokey:int, key2:timestamp, dt:timestamp
+ partitionColumnCount: 0
+ scratchColumnTypeNames: [string]
+ Map 2
+ Map Operator Tree:
+ TableScan
+ alias: item_dim_ts
+ filterExpr: key1 is not null (type: boolean)
+ Statistics: Num rows: 2 Data size: 448 Basic stats: COMPLETE Column stats: NONE
+ TableScan Vectorization:
+ native: true
+ vectorizationSchemaColumns: [0:key1:timestamp, 1:name:string, 2:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
+ Filter Operator
+ Filter Vectorization:
+ className: VectorFilterOperator
+ native: true
+ predicateExpression: SelectColumnIsNotNull(col 0:timestamp)
+ predicate: key1 is not null (type: boolean)
+ Statistics: Num rows: 2 Data size: 448 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: key1 (type: timestamp), name (type: string)
+ outputColumnNames: _col0, _col1
+ Select Vectorization:
+ className: VectorSelectOperator
+ native: true
+ projectedOutputColumnNums: [0, 1]
+ Statistics: Num rows: 2 Data size: 448 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: timestamp)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: timestamp)
+ Reduce Sink Vectorization:
+ className: VectorReduceSinkMultiKeyOperator
+ keyColumns: 0:timestamp
+ native: true
+ nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez 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
+ valueColumns: 1:string
+ Statistics: Num rows: 2 Data size: 448 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col1 (type: string)
+ Execution mode: vectorized, llap
+ LLAP IO: all inputs
+ Map Vectorization:
+ enabled: true
+ enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true
+ inputFormatFeatureSupport: [DECIMAL_64]
+ featureSupportInUse: [DECIMAL_64]
+ inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ allNative: true
+ usesVectorUDFAdaptor: false
+ vectorized: true
+ rowBatchContext:
+ dataColumnCount: 2
+ includeColumns: [0, 1]
+ dataColumns: key1:timestamp, name:string
+ partitionColumnCount: 0
+ scratchColumnTypeNames: []
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: select key1, key2, name, dt from orders_fact_ts join item_dim_ts on (orders_fact_ts.key2 = item_dim_ts.key1)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@item_dim_ts
+PREHOOK: Input: default@orders_fact_ts
+#### A masked pattern was here ####
+POSTHOOK: query: select key1, key2, name, dt from orders_fact_ts join item_dim_ts on (orders_fact_ts.key2 = item_dim_ts.key1)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@item_dim_ts
+POSTHOOK: Input: default@orders_fact_ts
+#### A masked pattern was here ####
+2002-01-30 00:00:00 2002-01-30 00:00:00 Item 102 2012-04-30 00:00:00
+2001-01-30 00:00:00 2001-01-30 00:00:00 Item 101 2011-01-30 00:00:00
diff --git a/ql/src/test/results/clientpositive/llap/vector_partitioned_date_time.q.out b/ql/src/test/results/clientpositive/llap/vector_partitioned_date_time.q.out
index 129eb7d..9889b42 100644
--- a/ql/src/test/results/clientpositive/llap/vector_partitioned_date_time.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_partitioned_date_time.q.out
@@ -495,7 +495,7 @@ STAGE PLANS:
sort order: +
Map-reduce partition columns: _col0 (type: date)
Reduce Sink Vectorization:
- className: VectorReduceSinkMultiKeyOperator
+ className: VectorReduceSinkLongOperator
native: true
nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez 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
Statistics: Num rows: 68 Data size: 4352 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1533,7 +1533,7 @@ STAGE PLANS:
sort order: +
Map-reduce partition columns: _col0 (type: date)
Reduce Sink Vectorization:
- className: VectorReduceSinkMultiKeyOperator
+ className: VectorReduceSinkLongOperator
native: true
nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez 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
Statistics: Num rows: 12 Data size: 768 Basic stats: COMPLETE Column stats: COMPLETE
@@ -3164,7 +3164,7 @@ STAGE PLANS:
sort order: +
Map-reduce partition columns: _col0 (type: date)
Reduce Sink Vectorization:
- className: VectorReduceSinkMultiKeyOperator
+ className: VectorReduceSinkLongOperator
native: true
nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez 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
Statistics: Num rows: 68 Data size: 4352 Basic stats: COMPLETE Column stats: COMPLETE
@@ -4202,7 +4202,7 @@ STAGE PLANS:
sort order: +
Map-reduce partition columns: _col0 (type: date)
Reduce Sink Vectorization:
- className: VectorReduceSinkMultiKeyOperator
+ className: VectorReduceSinkLongOperator
native: true
nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez 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
Statistics: Num rows: 12 Data size: 768 Basic stats: COMPLETE Column stats: COMPLETE