You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by kr...@apache.org on 2022/07/20 12:46:56 UTC
[hive] branch master updated: HIVE-26385: Iceberg integration: Implement merge into iceberg table (Krisztian Kasa, reviewed by Peter Vary)
This is an automated email from the ASF dual-hosted git repository.
krisztiankasa 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 8fdf802b1ff HIVE-26385: Iceberg integration: Implement merge into iceberg table (Krisztian Kasa, reviewed by Peter Vary)
8fdf802b1ff is described below
commit 8fdf802b1ffa34fffddcf10d44726d046179e883
Author: Krisztian Kasa <ka...@gmail.com>
AuthorDate: Wed Jul 20 14:46:49 2022 +0200
HIVE-26385: Iceberg integration: Implement merge into iceberg table (Krisztian Kasa, reviewed by Peter Vary)
---
.../mr/hive/HiveIcebergOutputCommitter.java | 8 +-
.../iceberg/mr/hive/writer/WriterBuilder.java | 5 +-
.../src/test/queries/positive/merge_iceberg_orc.q | 22 ++
.../positive/merge_iceberg_partitioned_orc.q | 22 ++
.../test/results/positive/merge_iceberg_orc.q.out | 321 ++++++++++++++++
.../positive/merge_iceberg_partitioned_orc.q.out | 349 +++++++++++++++++
.../hive/ql/parse/MergeSemanticAnalyzer.java | 59 +--
.../hive/ql/parse/RewriteSemanticAnalyzer.java | 124 ++++++-
.../ql/parse/UpdateDeleteSemanticAnalyzer.java | 51 +--
.../acid_direct_update_delete_with_merge.q.out | 2 +-
.../clientpositive/llap/acid_no_buckets.q.out | 168 ++++-----
.../clientpositive/llap/acid_subquery.q.out | 2 +-
.../clientpositive/llap/check_constraint.q.out | 42 +--
.../llap/create_transactional_full_acid.q.out | 2 +-
.../llap/dynamic_semijoin_reduction_3.q.out | 30 +-
.../llap/dynpart_sort_optimization_acid.q.out | 62 ++--
.../llap/enforce_constraint_notnull.q.out | 150 ++++----
.../llap/insert_into_default_keyword.q.out | 20 +-
.../results/clientpositive/llap/llap_acid.q.out | 2 +-
.../clientpositive/llap/llap_acid_fast.q.out | 2 +-
.../llap/masking_acid_no_masking.q.out | 2 +-
.../clientpositive/llap/runtime_stats_merge.q.out | 2 +-
.../clientpositive/llap/semijoin_hint.q.out | 122 +++---
.../results/clientpositive/llap/sort_acid.q.out | 10 +-
.../results/clientpositive/llap/sqlmerge.q.out | 30 +-
.../clientpositive/llap/sqlmerge_stats.q.out | 412 ++++++++++-----------
26 files changed, 1378 insertions(+), 643 deletions(-)
diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java
index 02ec4a1a913..5a394cad551 100644
--- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java
+++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java
@@ -70,6 +70,7 @@ import org.apache.iceberg.mr.hive.writer.WriterRegistry;
import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
import org.apache.iceberg.relocated.com.google.common.collect.Sets;
import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.iceberg.util.Tasks;
@@ -134,9 +135,14 @@ public class HiveIcebergOutputCommitter extends OutputCommitter {
String fileForCommitLocation = generateFileForCommitLocation(table.location(), jobConf,
attemptID.getJobID(), attemptID.getTaskID().getId());
if (writers.get(output) != null) {
+ Collection<DataFile> dataFiles = Lists.newArrayList();
+ Collection<DeleteFile> deleteFiles = Lists.newArrayList();
for (HiveIcebergWriter writer : writers.get(output)) {
- createFileForCommit(writer.files(), fileForCommitLocation, table.io());
+ FilesForCommit files = writer.files();
+ dataFiles.addAll(files.dataFiles());
+ deleteFiles.addAll(files.deleteFiles());
}
+ createFileForCommit(new FilesForCommit(dataFiles, deleteFiles), fileForCommitLocation, table.io());
} else {
LOG.info("CommitTask found no writer for specific table: {}, attemptID: {}", output, attemptID);
createFileForCommit(FilesForCommit.empty(), fileForCommitLocation, table.io());
diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/WriterBuilder.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/WriterBuilder.java
index b4b26738c42..ee0fd9a2f6a 100644
--- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/WriterBuilder.java
+++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/WriterBuilder.java
@@ -21,6 +21,7 @@ package org.apache.iceberg.mr.hive.writer;
import java.util.Locale;
import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
import org.apache.hadoop.hive.ql.Context.Operation;
import org.apache.hadoop.mapred.TaskAttemptID;
import org.apache.iceberg.FileFormat;
@@ -43,6 +44,8 @@ public class WriterBuilder {
private String queryId;
private int poolSize;
private Operation operation;
+ // A task may write multiple output files using multiple writers. Each of them must have a unique operationId.
+ private static AtomicInteger operationNum = new AtomicInteger(0);
private WriterBuilder(Table table) {
this.table = table;
@@ -95,7 +98,7 @@ public class WriterBuilder {
int currentSpecId = table.spec().specId();
int partitionId = attemptID.getTaskID().getId();
int taskId = attemptID.getId();
- String operationId = queryId + "-" + attemptID.getJobID();
+ String operationId = queryId + "-" + attemptID.getJobID() + "-" + operationNum.incrementAndGet();
OutputFileFactory outputFileFactory = OutputFileFactory.builderFor(table, partitionId, taskId)
.format(dataFileFormat)
.operationId("data-" + operationId)
diff --git a/iceberg/iceberg-handler/src/test/queries/positive/merge_iceberg_orc.q b/iceberg/iceberg-handler/src/test/queries/positive/merge_iceberg_orc.q
new file mode 100644
index 00000000000..203fdc2598c
--- /dev/null
+++ b/iceberg/iceberg-handler/src/test/queries/positive/merge_iceberg_orc.q
@@ -0,0 +1,22 @@
+-- SORT_QUERY_RESULTS
+set hive.explain.user=false;
+
+create external table target_ice(a int, b string, c int) stored by iceberg stored as orc tblproperties ('format-version'='2');
+create table source(a int, b string, c int);
+
+insert into target_ice values (1, 'one', 50), (2, 'two', 51), (111, 'one', 55), (333, 'two', 56);
+insert into source values (1, 'one', 50), (2, 'two', 51), (3, 'three', 52), (4, 'four', 53), (5, 'five', 54), (111, 'one', 55);
+
+-- merge
+explain
+merge into target_ice as t using source src ON t.a = src.a
+when matched and t.a > 100 THEN DELETE
+when matched then update set b = 'Merged', c = t.c + 10
+when not matched then insert values (src.a, src.b, src.c);
+
+merge into target_ice as t using source src ON t.a = src.a
+when matched and t.a > 100 THEN DELETE
+when matched then update set b = 'Merged', c = t.c + 10
+when not matched then insert values (src.a, src.b, src.c);
+
+select * from target_ice;
diff --git a/iceberg/iceberg-handler/src/test/queries/positive/merge_iceberg_partitioned_orc.q b/iceberg/iceberg-handler/src/test/queries/positive/merge_iceberg_partitioned_orc.q
new file mode 100644
index 00000000000..f5976162138
--- /dev/null
+++ b/iceberg/iceberg-handler/src/test/queries/positive/merge_iceberg_partitioned_orc.q
@@ -0,0 +1,22 @@
+-- SORT_QUERY_RESULTS
+set hive.explain.user=false;
+
+create external table target_ice(a int, b string, c int) partitioned by spec (bucket(16, a), truncate(3, b)) stored by iceberg stored as orc tblproperties ('format-version'='2');
+create table source(a int, b string, c int);
+
+insert into target_ice values (1, 'one', 50), (2, 'two', 51), (111, 'one', 55), (333, 'two', 56);
+insert into source values (1, 'one', 50), (2, 'two', 51), (3, 'three', 52), (4, 'four', 53), (5, 'five', 54), (111, 'one', 55);
+
+-- merge
+explain
+merge into target_ice as t using source src ON t.a = src.a
+when matched and t.a > 100 THEN DELETE
+when matched then update set b = 'Merged', c = t.c + 10
+when not matched then insert values (src.a, src.b, src.c);
+
+merge into target_ice as t using source src ON t.a = src.a
+when matched and t.a > 100 THEN DELETE
+when matched then update set b = 'Merged', c = t.c + 10
+when not matched then insert values (src.a, src.b, src.c);
+
+select * from target_ice;
diff --git a/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_orc.q.out b/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_orc.q.out
new file mode 100644
index 00000000000..8f314fefcbb
--- /dev/null
+++ b/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_orc.q.out
@@ -0,0 +1,321 @@
+PREHOOK: query: create external table target_ice(a int, b string, c int) stored by iceberg stored as orc tblproperties ('format-version'='2')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@target_ice
+POSTHOOK: query: create external table target_ice(a int, b string, c int) stored by iceberg stored as orc tblproperties ('format-version'='2')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@target_ice
+PREHOOK: query: create table source(a int, b string, c int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@source
+POSTHOOK: query: create table source(a int, b string, c int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@source
+PREHOOK: query: insert into target_ice values (1, 'one', 50), (2, 'two', 51), (111, 'one', 55), (333, 'two', 56)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@target_ice
+POSTHOOK: query: insert into target_ice values (1, 'one', 50), (2, 'two', 51), (111, 'one', 55), (333, 'two', 56)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@target_ice
+PREHOOK: query: insert into source values (1, 'one', 50), (2, 'two', 51), (3, 'three', 52), (4, 'four', 53), (5, 'five', 54), (111, 'one', 55)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@source
+POSTHOOK: query: insert into source values (1, 'one', 50), (2, 'two', 51), (3, 'three', 52), (4, 'four', 53), (5, 'five', 54), (111, 'one', 55)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@source
+POSTHOOK: Lineage: source.a SCRIPT []
+POSTHOOK: Lineage: source.b SCRIPT []
+POSTHOOK: Lineage: source.c SCRIPT []
+PREHOOK: query: explain
+merge into target_ice as t using source src ON t.a = src.a
+when matched and t.a > 100 THEN DELETE
+when matched then update set b = 'Merged', c = t.c + 10
+when not matched then insert values (src.a, src.b, src.c)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@source
+PREHOOK: Input: default@target_ice
+PREHOOK: Output: default@merge_tmp_table
+PREHOOK: Output: default@target_ice
+PREHOOK: Output: default@target_ice
+POSTHOOK: query: explain
+merge into target_ice as t using source src ON t.a = src.a
+when matched and t.a > 100 THEN DELETE
+when matched then update set b = 'Merged', c = t.c + 10
+when not matched then insert values (src.a, src.b, src.c)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@source
+POSTHOOK: Input: default@target_ice
+POSTHOOK: Output: default@merge_tmp_table
+POSTHOOK: Output: default@target_ice
+POSTHOOK: Output: default@target_ice
+STAGE DEPENDENCIES:
+ Stage-5 is a root stage
+ Stage-6 depends on stages: Stage-5
+ Stage-0 depends on stages: Stage-6
+ Stage-7 depends on stages: Stage-0
+ Stage-4 depends on stages: Stage-6
+ Stage-8 depends on stages: Stage-4
+
+STAGE PLANS:
+ Stage: Stage-5
+ Tez
+#### A masked pattern was here ####
+ Edges:
+ Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 6 (SIMPLE_EDGE)
+ Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+ Reducer 4 <- Reducer 2 (SIMPLE_EDGE)
+ Reducer 5 <- Reducer 2 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: src
+ Statistics: Num rows: 6 Data size: 576 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: a (type: int), b (type: string), c (type: int)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 6 Data size: 576 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: int)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 6 Data size: 576 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col1 (type: string), _col2 (type: int)
+ Map 6
+ Map Operator Tree:
+ TableScan
+ alias: target_ice
+ filterExpr: a is not null (type: boolean)
+ Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: a is not null (type: boolean)
+ Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint), b (type: string), c (type: int), a (type: int), c (type: int)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col5, _col6, _col7, _col8
+ Statistics: Num rows: 4 Data size: 1212 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col7 (type: int)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col7 (type: int)
+ Statistics: Num rows: 4 Data size: 1212 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col5 (type: string), _col6 (type: int), _col8 (type: int)
+ Reducer 2
+ Reduce Operator Tree:
+ Merge Join Operator
+ condition map:
+ Left Outer Join 0 to 1
+ keys:
+ 0 _col0 (type: int)
+ 1 _col7 (type: int)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
+ Statistics: Num rows: 10 Data size: 2515 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: _col4 (type: bigint), _col9 (type: int), _col3 (type: int), _col7 (type: int), _col1 (type: string), _col8 (type: string), _col0 (type: int), _col2 (type: int), _col6 (type: bigint), _col5 (type: string), _col11 (type: int), _col10 (type: int)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
+ Statistics: Num rows: 10 Data size: 2475 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: ((_col11 = _col6) and (_col11 > 100)) (type: boolean)
+ Statistics: Num rows: 1 Data size: 399 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: _col2 (type: int), _col0 (type: bigint), _col9 (type: string), _col8 (type: bigint), _col3 (type: int), _col5 (type: string), _col1 (type: int)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
+ Statistics: Num rows: 1 Data size: 299 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint)
+ null sort order: zzzz
+ sort order: ++++
+ Statistics: Num rows: 1 Data size: 299 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col4 (type: int), _col5 (type: string), _col6 (type: int)
+ Filter Operator
+ predicate: ((_col11 = _col6) and (_col11 <= 100)) (type: boolean)
+ Statistics: Num rows: 1 Data size: 399 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: _col2 (type: int), _col0 (type: bigint), _col9 (type: string), _col8 (type: bigint), _col3 (type: int), _col5 (type: string), _col1 (type: int)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
+ Statistics: Num rows: 1 Data size: 299 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint)
+ null sort order: zzzz
+ sort order: ++++
+ Statistics: Num rows: 1 Data size: 299 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col4 (type: int), _col5 (type: string), _col6 (type: int)
+ Filter Operator
+ predicate: ((_col11 = _col6) and (_col11 <= 100)) (type: boolean)
+ Statistics: Num rows: 1 Data size: 399 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: _col11 (type: int), 'Merged' (type: string), (_col10 + 10) (type: int)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: COMPLETE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: COMPLETE
+ table:
+ input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
+ output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat
+ serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe
+ name: default.target_ice
+ Filter Operator
+ predicate: _col11 is null (type: boolean)
+ Statistics: Num rows: 5 Data size: 1389 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: _col6 (type: int), _col4 (type: string), _col7 (type: int)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 5 Data size: 480 Basic stats: COMPLETE Column stats: COMPLETE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 5 Data size: 480 Basic stats: COMPLETE Column stats: COMPLETE
+ table:
+ input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
+ output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat
+ serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe
+ name: default.target_ice
+ Filter Operator
+ predicate: (_col11 = _col6) (type: boolean)
+ Statistics: Num rows: 5 Data size: 1389 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: _col0 (type: bigint), _col2 (type: int), _col8 (type: bigint), _col9 (type: string)
+ outputColumnNames: _col0, _col2, _col8, _col9
+ Statistics: Num rows: 5 Data size: 1389 Basic stats: COMPLETE Column stats: COMPLETE
+ Group By Operator
+ aggregations: count()
+ keys: _col2 (type: int), _col0 (type: bigint), _col9 (type: string), _col8 (type: bigint)
+ minReductionHashAggr: 0.4
+ mode: hash
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4
+ Statistics: Num rows: 4 Data size: 644 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint)
+ null sort order: zzzz
+ sort order: ++++
+ Map-reduce partition columns: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint)
+ Statistics: Num rows: 4 Data size: 644 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col4 (type: bigint)
+ Reducer 3
+ Reduce Operator Tree:
+ Select Operator
+ expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: bigint), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: bigint), VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
+ Statistics: Num rows: 1 Data size: 299 Basic stats: COMPLETE Column stats: COMPLETE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 299 Basic stats: COMPLETE Column stats: COMPLETE
+ table:
+ input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
+ output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat
+ serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe
+ name: default.target_ice
+ Reducer 4
+ Reduce Operator Tree:
+ Select Operator
+ expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: bigint), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: bigint), VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
+ Statistics: Num rows: 1 Data size: 299 Basic stats: COMPLETE Column stats: COMPLETE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 299 Basic stats: COMPLETE Column stats: COMPLETE
+ table:
+ input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
+ output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat
+ serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe
+ name: default.target_ice
+ Reducer 5
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations: count(VALUE._col0)
+ keys: KEY._col0 (type: int), KEY._col1 (type: bigint), KEY._col2 (type: string), KEY._col3 (type: bigint)
+ mode: mergepartial
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4
+ Statistics: Num rows: 4 Data size: 644 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: (_col4 > 1L) (type: boolean)
+ Statistics: Num rows: 1 Data size: 212 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: cardinality_violation(_col0,_col1,_col2,_col3) (type: int)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ name: default.merge_tmp_table
+
+ Stage: Stage-6
+ Dependency Collection
+
+ Stage: Stage-0
+ Move Operator
+ tables:
+ replace: false
+ table:
+ input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
+ output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat
+ serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe
+ name: default.target_ice
+
+ Stage: Stage-7
+ Stats Work
+ Basic Stats Work:
+
+ Stage: Stage-4
+ Move Operator
+ tables:
+ replace: false
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ name: default.merge_tmp_table
+
+ Stage: Stage-8
+ Stats Work
+ Basic Stats Work:
+
+PREHOOK: query: merge into target_ice as t using source src ON t.a = src.a
+when matched and t.a > 100 THEN DELETE
+when matched then update set b = 'Merged', c = t.c + 10
+when not matched then insert values (src.a, src.b, src.c)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@source
+PREHOOK: Input: default@target_ice
+PREHOOK: Output: default@merge_tmp_table
+PREHOOK: Output: default@target_ice
+PREHOOK: Output: default@target_ice
+POSTHOOK: query: merge into target_ice as t using source src ON t.a = src.a
+when matched and t.a > 100 THEN DELETE
+when matched then update set b = 'Merged', c = t.c + 10
+when not matched then insert values (src.a, src.b, src.c)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@source
+POSTHOOK: Input: default@target_ice
+POSTHOOK: Output: default@merge_tmp_table
+POSTHOOK: Output: default@target_ice
+POSTHOOK: Output: default@target_ice
+POSTHOOK: Lineage: merge_tmp_table.val EXPRESSION [(target_ice)target_ice.null, ]
+PREHOOK: query: select * from target_ice
+PREHOOK: type: QUERY
+PREHOOK: Input: default@target_ice
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from target_ice
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@target_ice
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+1 Merged 60
+2 Merged 61
+3 three 52
+333 two 56
+4 four 53
+5 five 54
diff --git a/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_partitioned_orc.q.out b/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_partitioned_orc.q.out
new file mode 100644
index 00000000000..0ae5eda9c79
--- /dev/null
+++ b/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_partitioned_orc.q.out
@@ -0,0 +1,349 @@
+PREHOOK: query: create external table target_ice(a int, b string, c int) partitioned by spec (bucket(16, a), truncate(3, b)) stored by iceberg stored as orc tblproperties ('format-version'='2')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@target_ice
+POSTHOOK: query: create external table target_ice(a int, b string, c int) partitioned by spec (bucket(16, a), truncate(3, b)) stored by iceberg stored as orc tblproperties ('format-version'='2')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@target_ice
+PREHOOK: query: create table source(a int, b string, c int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@source
+POSTHOOK: query: create table source(a int, b string, c int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@source
+PREHOOK: query: insert into target_ice values (1, 'one', 50), (2, 'two', 51), (111, 'one', 55), (333, 'two', 56)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@target_ice
+POSTHOOK: query: insert into target_ice values (1, 'one', 50), (2, 'two', 51), (111, 'one', 55), (333, 'two', 56)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@target_ice
+PREHOOK: query: insert into source values (1, 'one', 50), (2, 'two', 51), (3, 'three', 52), (4, 'four', 53), (5, 'five', 54), (111, 'one', 55)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@source
+POSTHOOK: query: insert into source values (1, 'one', 50), (2, 'two', 51), (3, 'three', 52), (4, 'four', 53), (5, 'five', 54), (111, 'one', 55)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@source
+POSTHOOK: Lineage: source.a SCRIPT []
+POSTHOOK: Lineage: source.b SCRIPT []
+POSTHOOK: Lineage: source.c SCRIPT []
+PREHOOK: query: explain
+merge into target_ice as t using source src ON t.a = src.a
+when matched and t.a > 100 THEN DELETE
+when matched then update set b = 'Merged', c = t.c + 10
+when not matched then insert values (src.a, src.b, src.c)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@source
+PREHOOK: Input: default@target_ice
+PREHOOK: Output: default@merge_tmp_table
+PREHOOK: Output: default@target_ice
+PREHOOK: Output: default@target_ice
+POSTHOOK: query: explain
+merge into target_ice as t using source src ON t.a = src.a
+when matched and t.a > 100 THEN DELETE
+when matched then update set b = 'Merged', c = t.c + 10
+when not matched then insert values (src.a, src.b, src.c)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@source
+POSTHOOK: Input: default@target_ice
+POSTHOOK: Output: default@merge_tmp_table
+POSTHOOK: Output: default@target_ice
+POSTHOOK: Output: default@target_ice
+STAGE DEPENDENCIES:
+ Stage-5 is a root stage
+ Stage-6 depends on stages: Stage-5
+ Stage-0 depends on stages: Stage-6
+ Stage-7 depends on stages: Stage-0
+ Stage-4 depends on stages: Stage-6
+ Stage-8 depends on stages: Stage-4
+
+STAGE PLANS:
+ Stage: Stage-5
+ Tez
+#### A masked pattern was here ####
+ Edges:
+ Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 8 (SIMPLE_EDGE)
+ Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+ Reducer 4 <- Reducer 2 (SIMPLE_EDGE)
+ Reducer 5 <- Reducer 2 (SIMPLE_EDGE)
+ Reducer 6 <- Reducer 2 (SIMPLE_EDGE)
+ Reducer 7 <- Reducer 2 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: src
+ Statistics: Num rows: 6 Data size: 576 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: a (type: int), b (type: string), c (type: int)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 6 Data size: 576 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: int)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 6 Data size: 576 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col1 (type: string), _col2 (type: int)
+ Map 8
+ Map Operator Tree:
+ TableScan
+ alias: target_ice
+ filterExpr: a is not null (type: boolean)
+ Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: a is not null (type: boolean)
+ Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint), b (type: string), c (type: int), a (type: int), c (type: int)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col5, _col6, _col7, _col8
+ Statistics: Num rows: 4 Data size: 1212 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col7 (type: int)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col7 (type: int)
+ Statistics: Num rows: 4 Data size: 1212 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col5 (type: string), _col6 (type: int), _col8 (type: int)
+ Reducer 2
+ Reduce Operator Tree:
+ Merge Join Operator
+ condition map:
+ Left Outer Join 0 to 1
+ keys:
+ 0 _col0 (type: int)
+ 1 _col7 (type: int)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
+ Statistics: Num rows: 10 Data size: 2515 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: _col4 (type: bigint), _col9 (type: int), _col3 (type: int), _col7 (type: int), _col1 (type: string), _col8 (type: string), _col0 (type: int), _col2 (type: int), _col6 (type: bigint), _col5 (type: string), _col11 (type: int), _col10 (type: int)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
+ Statistics: Num rows: 10 Data size: 2475 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: ((_col11 = _col6) and (_col11 > 100)) (type: boolean)
+ Statistics: Num rows: 1 Data size: 399 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: _col2 (type: int), _col0 (type: bigint), _col9 (type: string), _col8 (type: bigint), _col3 (type: int), _col5 (type: string), _col1 (type: int)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
+ Statistics: Num rows: 1 Data size: 299 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint)
+ null sort order: zzzz
+ sort order: ++++
+ Statistics: Num rows: 1 Data size: 299 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col4 (type: int), _col5 (type: string), _col6 (type: int)
+ Filter Operator
+ predicate: ((_col11 = _col6) and (_col11 <= 100)) (type: boolean)
+ Statistics: Num rows: 1 Data size: 399 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: _col2 (type: int), _col0 (type: bigint), _col9 (type: string), _col8 (type: bigint), _col3 (type: int), _col5 (type: string), _col1 (type: int)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
+ Statistics: Num rows: 1 Data size: 299 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint)
+ null sort order: zzzz
+ sort order: ++++
+ Statistics: Num rows: 1 Data size: 299 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col4 (type: int), _col5 (type: string), _col6 (type: int)
+ Filter Operator
+ predicate: ((_col11 = _col6) and (_col11 <= 100)) (type: boolean)
+ Statistics: Num rows: 1 Data size: 399 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: _col11 (type: int), 'Merged' (type: string), (_col10 + 10) (type: int)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: iceberg_bucket(_col0, 16) (type: int), _col1 (type: string)
+ null sort order: aa
+ sort order: ++
+ Map-reduce partition columns: iceberg_bucket(_col0, 16) (type: int), _col1 (type: string)
+ Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int)
+ Filter Operator
+ predicate: _col11 is null (type: boolean)
+ Statistics: Num rows: 5 Data size: 1389 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: _col6 (type: int), _col4 (type: string), _col7 (type: int)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 5 Data size: 480 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: iceberg_bucket(_col0, 16) (type: int), _col1 (type: string)
+ null sort order: aa
+ sort order: ++
+ Map-reduce partition columns: iceberg_bucket(_col0, 16) (type: int), _col1 (type: string)
+ Statistics: Num rows: 5 Data size: 480 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int)
+ Filter Operator
+ predicate: (_col11 = _col6) (type: boolean)
+ Statistics: Num rows: 5 Data size: 1389 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: _col0 (type: bigint), _col2 (type: int), _col8 (type: bigint), _col9 (type: string)
+ outputColumnNames: _col0, _col2, _col8, _col9
+ Statistics: Num rows: 5 Data size: 1389 Basic stats: COMPLETE Column stats: COMPLETE
+ Group By Operator
+ aggregations: count()
+ keys: _col2 (type: int), _col0 (type: bigint), _col9 (type: string), _col8 (type: bigint)
+ minReductionHashAggr: 0.4
+ mode: hash
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4
+ Statistics: Num rows: 4 Data size: 644 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint)
+ null sort order: zzzz
+ sort order: ++++
+ Map-reduce partition columns: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint)
+ Statistics: Num rows: 4 Data size: 644 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col4 (type: bigint)
+ Reducer 3
+ Reduce Operator Tree:
+ Select Operator
+ expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: bigint), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: bigint), VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
+ Statistics: Num rows: 1 Data size: 299 Basic stats: COMPLETE Column stats: COMPLETE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 299 Basic stats: COMPLETE Column stats: COMPLETE
+ table:
+ input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
+ output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat
+ serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe
+ name: default.target_ice
+ Reducer 4
+ Reduce Operator Tree:
+ Select Operator
+ expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: bigint), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: bigint), VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
+ Statistics: Num rows: 1 Data size: 299 Basic stats: COMPLETE Column stats: COMPLETE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 299 Basic stats: COMPLETE Column stats: COMPLETE
+ table:
+ input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
+ output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat
+ serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe
+ name: default.target_ice
+ Reducer 5
+ Reduce Operator Tree:
+ Select Operator
+ expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int), KEY.iceberg_bucket(_col0, 16) (type: int), KEY._col1 (type: string)
+ outputColumnNames: _col0, _col1, _col2, iceberg_bucket(_col0, 16), _col1
+ File Output Operator
+ compressed: false
+ Dp Sort State: PARTITION_SORTED
+ Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: COMPLETE
+ table:
+ input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
+ output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat
+ serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe
+ name: default.target_ice
+ Reducer 6
+ Reduce Operator Tree:
+ Select Operator
+ expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int), KEY.iceberg_bucket(_col0, 16) (type: int), KEY._col1 (type: string)
+ outputColumnNames: _col0, _col1, _col2, iceberg_bucket(_col0, 16), _col1
+ File Output Operator
+ compressed: false
+ Dp Sort State: PARTITION_SORTED
+ Statistics: Num rows: 5 Data size: 480 Basic stats: COMPLETE Column stats: COMPLETE
+ table:
+ input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
+ output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat
+ serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe
+ name: default.target_ice
+ Reducer 7
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations: count(VALUE._col0)
+ keys: KEY._col0 (type: int), KEY._col1 (type: bigint), KEY._col2 (type: string), KEY._col3 (type: bigint)
+ mode: mergepartial
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4
+ Statistics: Num rows: 4 Data size: 644 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: (_col4 > 1L) (type: boolean)
+ Statistics: Num rows: 1 Data size: 212 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: cardinality_violation(_col0,_col1,_col2,_col3) (type: int)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ name: default.merge_tmp_table
+
+ Stage: Stage-6
+ Dependency Collection
+
+ Stage: Stage-0
+ Move Operator
+ tables:
+ replace: false
+ table:
+ input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
+ output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat
+ serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe
+ name: default.target_ice
+
+ Stage: Stage-7
+ Stats Work
+ Basic Stats Work:
+
+ Stage: Stage-4
+ Move Operator
+ tables:
+ replace: false
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ name: default.merge_tmp_table
+
+ Stage: Stage-8
+ Stats Work
+ Basic Stats Work:
+
+PREHOOK: query: merge into target_ice as t using source src ON t.a = src.a
+when matched and t.a > 100 THEN DELETE
+when matched then update set b = 'Merged', c = t.c + 10
+when not matched then insert values (src.a, src.b, src.c)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@source
+PREHOOK: Input: default@target_ice
+PREHOOK: Output: default@merge_tmp_table
+PREHOOK: Output: default@target_ice
+PREHOOK: Output: default@target_ice
+POSTHOOK: query: merge into target_ice as t using source src ON t.a = src.a
+when matched and t.a > 100 THEN DELETE
+when matched then update set b = 'Merged', c = t.c + 10
+when not matched then insert values (src.a, src.b, src.c)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@source
+POSTHOOK: Input: default@target_ice
+POSTHOOK: Output: default@merge_tmp_table
+POSTHOOK: Output: default@target_ice
+POSTHOOK: Output: default@target_ice
+POSTHOOK: Lineage: merge_tmp_table.val EXPRESSION [(target_ice)target_ice.null, ]
+PREHOOK: query: select * from target_ice
+PREHOOK: type: QUERY
+PREHOOK: Input: default@target_ice
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from target_ice
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@target_ice
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+1 Merged 60
+2 Merged 61
+3 three 52
+333 two 56
+4 four 53
+5 five 54
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/MergeSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/MergeSemanticAnalyzer.java
index 4532c416f7e..b5d58c490a8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/MergeSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/MergeSemanticAnalyzer.java
@@ -21,13 +21,11 @@ import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
-import java.util.IdentityHashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
-import org.antlr.runtime.TokenRewriteStream;
-import org.apache.commons.collections.MapUtils;
+import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hadoop.hive.metastore.Warehouse;
@@ -36,6 +34,7 @@ import org.apache.hadoop.hive.metastore.api.MetaException;
import org.apache.hadoop.hive.ql.Context;
import org.apache.hadoop.hive.ql.ErrorMsg;
import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.io.AcidUtils;
import org.apache.hadoop.hive.ql.lib.Node;
import org.apache.hadoop.hive.ql.metadata.HiveException;
import org.apache.hadoop.hive.ql.metadata.HiveUtils;
@@ -137,7 +136,18 @@ public class MergeSemanticAnalyzer extends RewriteSemanticAnalyzer {
List<ASTNode> whenClauses = findWhenClauses(tree, whenClauseBegins);
StringBuilder rewrittenQueryStr = createRewrittenQueryStrBuilder();
- appendTarget(rewrittenQueryStr, targetNameNode, targetName);
+ rewrittenQueryStr.append("(SELECT ");
+ boolean nonNativeAcid = AcidUtils.isNonNativeAcidTable(targetTable);
+ String subQueryAlias = isAliased(targetNameNode) ? targetName : targetTable.getTTable().getTableName();
+ ColumnAppender columnAppender = getColumnAppender(subQueryAlias);
+ columnAppender.appendAcidSelectColumns(rewrittenQueryStr, Context.Operation.UPDATE);
+
+ rewrittenQueryStr.deleteCharAt(rewrittenQueryStr.length() - 1); // remove last ','
+ addColsToSelect(targetTable.getCols(), rewrittenQueryStr);
+ addColsToSelect(targetTable.getPartCols(), rewrittenQueryStr);
+ rewrittenQueryStr.append(" FROM ").append(getFullTableNameForSQL(targetNameNode)).append(") ");
+ rewrittenQueryStr.append(subQueryAlias);
+ rewrittenQueryStr.append('\n');
rewrittenQueryStr.append(INDENT).append(chooseJoinType(whenClauses)).append("\n");
if (source.getType() == HiveParser.TOK_SUBQUERY) {
@@ -158,7 +168,8 @@ public class MergeSemanticAnalyzer extends RewriteSemanticAnalyzer {
hintStr = " /*+ " + qHint.getText() + " */ ";
}
final boolean splitUpdateEarly = HiveConf.getBoolVar(conf, HiveConf.ConfVars.SPLIT_UPDATE) ||
- HiveConf.getBoolVar(conf, HiveConf.ConfVars.MERGE_SPLIT_UPDATE);
+ HiveConf.getBoolVar(conf, HiveConf.ConfVars.MERGE_SPLIT_UPDATE) ||
+ nonNativeAcid;
/**
* We allow at most 2 WHEN MATCHED clause, in which case 1 must be Update the other Delete
* If we have both update and delete, the 1st one (in SQL code) must have "AND <extra predicate>"
@@ -182,7 +193,7 @@ public class MergeSemanticAnalyzer extends RewriteSemanticAnalyzer {
numWhenMatchedUpdateClauses++;
String s = handleUpdate(whenClause, rewrittenQueryStr, targetNameNode,
onClauseAsText, targetTable, extraPredicate, hintProcessed ? null : hintStr,
- splitUpdateEarly);
+ splitUpdateEarly, columnAppender);
hintProcessed = true;
if (numWhenMatchedUpdateClauses + numWhenMatchedDeleteClauses == 1) {
extraPredicate = s; //i.e. it's the 1st WHEN MATCHED
@@ -190,8 +201,8 @@ public class MergeSemanticAnalyzer extends RewriteSemanticAnalyzer {
break;
case HiveParser.TOK_DELETE:
numWhenMatchedDeleteClauses++;
- String s1 = handleDelete(whenClause, rewrittenQueryStr, targetNameNode,
- onClauseAsText, extraPredicate, hintProcessed ? null : hintStr, false);
+ String s1 = handleDelete(whenClause, rewrittenQueryStr,
+ onClauseAsText, extraPredicate, hintProcessed ? null : hintStr, false, columnAppender);
hintProcessed = true;
if (numWhenMatchedUpdateClauses + numWhenMatchedDeleteClauses == 1) {
extraPredicate = s1; //i.e. it's the 1st WHEN MATCHED
@@ -214,7 +225,7 @@ public class MergeSemanticAnalyzer extends RewriteSemanticAnalyzer {
}
boolean validating = handleCardinalityViolation(rewrittenQueryStr, targetNameNode, onClauseAsText, targetTable,
- numWhenMatchedDeleteClauses == 0 && numWhenMatchedUpdateClauses == 0);
+ numWhenMatchedDeleteClauses == 0 && numWhenMatchedUpdateClauses == 0, columnAppender);
ReparseResult rr = parseRewrittenQuery(rewrittenQueryStr, ctx.getCmd());
Context rewrittenCtx = rr.rewrittenCtx;
ASTNode rewrittenTree = rr.rewrittenTree;
@@ -283,7 +294,7 @@ public class MergeSemanticAnalyzer extends RewriteSemanticAnalyzer {
* @return true if another Insert clause was added
*/
private boolean handleCardinalityViolation(StringBuilder rewrittenQueryStr, ASTNode target,
- String onClauseAsString, Table targetTable, boolean onlyHaveWhenNotMatchedClause)
+ String onClauseAsString, Table targetTable, boolean onlyHaveWhenNotMatchedClause, ColumnAppender columnAppender)
throws SemanticException {
if (!conf.getBoolVar(HiveConf.ConfVars.MERGE_CARDINALITY_VIOLATION_CHECK)) {
LOG.info("Merge statement cardinality violation check is disabled: " +
@@ -297,15 +308,16 @@ public class MergeSemanticAnalyzer extends RewriteSemanticAnalyzer {
//this is a tmp table and thus Session scoped and acid requires SQL statement to be serial in a
// given session, i.e. the name can be fixed across all invocations
String tableName = "merge_tmp_table";
+ List<String> sortKeys = columnAppender.getSortKeys();
rewrittenQueryStr.append("INSERT INTO ").append(tableName)
.append("\n SELECT cardinality_violation(")
- .append(getSimpleTableName(target)).append(".ROW__ID");
- addPartitionColsToSelect(targetTable.getPartCols(), rewrittenQueryStr, target);
+ .append(StringUtils.join(sortKeys, ","));
+ addColsToSelect(targetTable.getPartCols(), rewrittenQueryStr, target);
rewrittenQueryStr.append(")\n WHERE ").append(onClauseAsString)
- .append(" GROUP BY ").append(getSimpleTableName(target)).append(".ROW__ID");
+ .append(" GROUP BY ").append(StringUtils.join(sortKeys, ","));
- addPartitionColsToSelect(targetTable.getPartCols(), rewrittenQueryStr, target);
+ addColsToSelect(targetTable.getPartCols(), rewrittenQueryStr, target);
rewrittenQueryStr.append(" HAVING count(*) > 1");
//say table T has partition p, we are generating
@@ -339,7 +351,8 @@ public class MergeSemanticAnalyzer extends RewriteSemanticAnalyzer {
* @param deleteExtraPredicate - see notes at caller
*/
private String handleUpdate(ASTNode whenMatchedUpdateClause, StringBuilder rewrittenQueryStr, ASTNode target,
- String onClauseAsString, Table targetTable, String deleteExtraPredicate, String hintStr, boolean splitUpdateEarly)
+ String onClauseAsString, Table targetTable, String deleteExtraPredicate, String hintStr,
+ boolean splitUpdateEarly, ColumnAppender columnAppender)
throws SemanticException {
assert whenMatchedUpdateClause.getType() == HiveParser.TOK_MATCHED;
assert getWhenClauseOperation(whenMatchedUpdateClause).getType() == HiveParser.TOK_UPDATE;
@@ -419,8 +432,8 @@ public class MergeSemanticAnalyzer extends RewriteSemanticAnalyzer {
* by this call to handleDelete()
*/
rewrittenQueryStr.append(" -- update clause (delete part)\n");
- handleDelete(whenMatchedUpdateClause, rewrittenQueryStr, target, onClauseAsString,
- deleteExtraPredicate, hintStr, true);
+ handleDelete(whenMatchedUpdateClause, rewrittenQueryStr, onClauseAsString,
+ deleteExtraPredicate, hintStr, true, columnAppender);
}
return extraPredicate;
@@ -431,15 +444,15 @@ public class MergeSemanticAnalyzer extends RewriteSemanticAnalyzer {
* @param updateExtraPredicate - see notes at caller
*/
private String handleDelete(ASTNode whenMatchedDeleteClause, StringBuilder rewrittenQueryStr,
- ASTNode target, String onClauseAsString, String updateExtraPredicate,
- String hintStr, boolean splitUpdateEarly) throws SemanticException {
+ String onClauseAsString, String updateExtraPredicate,
+ String hintStr, boolean splitUpdateEarly, ColumnAppender columnAppender) {
assert whenMatchedDeleteClause.getType() == HiveParser.TOK_MATCHED;
assert (splitUpdateEarly &&
getWhenClauseOperation(whenMatchedDeleteClause).getType() == HiveParser.TOK_UPDATE) ||
getWhenClauseOperation(whenMatchedDeleteClause).getType() == HiveParser.TOK_DELETE;
- String targetName = getSimpleTableName(target);
- appendDeleteBranch(rewrittenQueryStr, hintStr, targetName, Collections.singletonList(targetName + ".ROW__ID"));
+ List<String> deleteValues = columnAppender.getDeleteValues(Context.Operation.DELETE);
+ appendInsertBranch(rewrittenQueryStr, hintStr, deleteValues);
rewrittenQueryStr.append(INDENT).append("WHERE ").append(onClauseAsString);
String extraPredicate = getWhenClausePredicate(whenMatchedDeleteClause);
@@ -450,7 +463,9 @@ public class MergeSemanticAnalyzer extends RewriteSemanticAnalyzer {
if (updateExtraPredicate != null) {
rewrittenQueryStr.append(" AND NOT(").append(updateExtraPredicate).append(")");
}
- appendSortBy(rewrittenQueryStr, Collections.singletonList(targetName + ".ROW__ID "));
+ List<String> sortKeys = columnAppender.getSortKeys();
+ rewrittenQueryStr.append("\n").append(INDENT);
+ appendSortBy(rewrittenQueryStr, sortKeys);
return extraPredicate;
}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/RewriteSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/RewriteSemanticAnalyzer.java
index f934990b137..d38463a37a3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/RewriteSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/RewriteSemanticAnalyzer.java
@@ -23,6 +23,7 @@ import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
+import java.util.stream.Collectors;
import org.antlr.runtime.TokenRewriteStream;
import org.apache.commons.lang3.StringUtils;
@@ -48,6 +49,7 @@ import org.apache.hadoop.hive.ql.metadata.VirtualColumn;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import static java.util.Collections.singletonList;
import static org.apache.commons.lang3.StringUtils.isNotBlank;
@@ -112,7 +114,7 @@ public abstract class RewriteSemanticAnalyzer extends CalcitePlanner {
* Append list of partition columns to Insert statement, i.e. the 2nd set of partCol1,partCol2
* INSERT INTO T PARTITION(partCol1,partCol2...) SELECT col1, ... partCol1,partCol2...
*/
- protected void addPartitionColsToSelect(List<FieldSchema> partCols, StringBuilder rewrittenQueryStr)
+ protected void addColsToSelect(List<FieldSchema> partCols, StringBuilder rewrittenQueryStr)
throws SemanticException {
// If the table is partitioned, we need to select the partition columns as well.
if (partCols != null) {
@@ -124,24 +126,21 @@ public abstract class RewriteSemanticAnalyzer extends CalcitePlanner {
}
/**
- * Append list of partition columns to Insert statement, i.e. the 2nd set of partCol1,partCol2
- * INSERT INTO T PARTITION(partCol1,partCol2...) SELECT col1, ... partCol1,partCol2...
- * @param target target table
+ * Append list of columns to rewritten statement.
*/
- protected void addPartitionColsToSelect(List<FieldSchema> partCols, StringBuilder rewrittenQueryStr,
- ASTNode target) throws SemanticException {
- addPartitionColsToSelect(partCols, rewrittenQueryStr, getSimpleTableName(target));
+ protected void addColsToSelect(List<FieldSchema> cols, StringBuilder rewrittenQueryStr,
+ ASTNode target) throws SemanticException {
+ addColsToSelect(cols, rewrittenQueryStr, getSimpleTableName(target));
}
/**
- * Append list of partition columns to Insert statement, i.e. the 2nd set of partCol1,partCol2
- * INSERT INTO T PARTITION(partCol1,partCol2...) SELECT col1, ... partCol1,partCol2...
- * @param alias table name or alias
+ * Append list of columns to rewritten statement.
+ * Column names are qualified with the specified alias and quoted.
*/
- protected void addPartitionColsToSelect(List<FieldSchema> partCols, StringBuilder rewrittenQueryStr, String alias) {
+ protected void addColsToSelect(List<FieldSchema> cols, StringBuilder rewrittenQueryStr, String alias) {
// If the table is partitioned, we need to select the partition columns as well.
- if (partCols != null) {
- for (FieldSchema fschema : partCols) {
+ if (cols != null) {
+ for (FieldSchema fschema : cols) {
rewrittenQueryStr.append(", ");
rewrittenQueryStr.append(alias).append('.');
rewrittenQueryStr.append(HiveUtils.unparseIdentifier(fschema.getName(), this.conf));
@@ -613,4 +612,103 @@ public abstract class RewriteSemanticAnalyzer extends CalcitePlanner {
return ctx.getTokenRewriteStream().toString(n.getTokenStartIndex(),
n.getTokenStopIndex() + 1).trim();
}
+
+ public static final String DELETE_PREFIX = "__d__";
+ public static final String SUB_QUERY_ALIAS = "s";
+
+ protected ColumnAppender getColumnAppender(String subQueryAlias) {
+ boolean nonNativeAcid = AcidUtils.isNonNativeAcidTable(targetTable);
+ return nonNativeAcid ? new NonNativeAcidColumnAppender(targetTable, conf, subQueryAlias) :
+ new NativeAcidColumnAppender(targetTable, conf, subQueryAlias);
+ }
+
+ protected interface ColumnAppender {
+ void appendAcidSelectColumns(StringBuilder stringBuilder, Context.Operation operation);
+ List<String> getDeleteValues(Context.Operation operation);
+ List<String> getSortKeys();
+ }
+
+ protected static class NativeAcidColumnAppender implements ColumnAppender {
+
+ private final Table table;
+ private final HiveConf conf;
+ private final String subQueryAlias;
+
+ public NativeAcidColumnAppender(Table table, HiveConf conf, String subQueryAlias) {
+ this.table = table;
+ this.conf = conf;
+ this.subQueryAlias = subQueryAlias;
+ }
+
+ @Override
+ public void appendAcidSelectColumns(StringBuilder stringBuilder, Context.Operation operation) {
+ stringBuilder.append("ROW__ID,");
+ for (FieldSchema fieldSchema : table.getPartCols()) {
+ String identifier = HiveUtils.unparseIdentifier(fieldSchema.getName(), this.conf);
+ stringBuilder.append(identifier);
+ stringBuilder.append(",");
+ }
+ }
+
+ @Override
+ public List<String> getDeleteValues(Context.Operation operation) {
+ List<String> deleteValues = new ArrayList<>(1 + table.getPartCols().size());
+ deleteValues.add(subQueryAlias + ".ROW__ID");
+ for (FieldSchema fieldSchema : table.getPartCols()) {
+ deleteValues.add(subQueryAlias + "." + HiveUtils.unparseIdentifier(fieldSchema.getName(), conf));
+ }
+ return deleteValues;
+ }
+
+ @Override
+ public List<String> getSortKeys() {
+ return singletonList(subQueryAlias + ".ROW__ID ");
+ }
+ }
+
+ protected static class NonNativeAcidColumnAppender implements ColumnAppender {
+
+ private final Table table;
+ private final HiveConf conf;
+ private final String subQueryAlias;
+
+ public NonNativeAcidColumnAppender(Table table, HiveConf conf, String subQueryAlias) {
+ this.table = table;
+ this.conf = conf;
+ this.subQueryAlias = subQueryAlias;
+ }
+
+ @Override
+ public void appendAcidSelectColumns(StringBuilder stringBuilder, Context.Operation operation) {
+ List<FieldSchema> acidSelectColumns = table.getStorageHandler().acidSelectColumns(table, operation);
+ for (FieldSchema fieldSchema : acidSelectColumns) {
+ String identifier = HiveUtils.unparseIdentifier(fieldSchema.getName(), this.conf);
+ stringBuilder.append(identifier).append(" AS ");
+ String prefixedIdentifier = HiveUtils.unparseIdentifier(DELETE_PREFIX + fieldSchema.getName(), this.conf);
+ stringBuilder.append(prefixedIdentifier);
+ stringBuilder.append(",");
+ }
+ }
+
+ @Override
+ public List<String> getDeleteValues(Context.Operation operation) {
+ List<FieldSchema> acidSelectColumns = table.getStorageHandler().acidSelectColumns(table, operation);
+ List<String> deleteValues = new ArrayList<>(acidSelectColumns.size());
+ for (FieldSchema fieldSchema : acidSelectColumns) {
+ String prefixedIdentifier = HiveUtils.unparseIdentifier(DELETE_PREFIX + fieldSchema.getName(), this.conf);
+ deleteValues.add(String.format("%s.%s", subQueryAlias, prefixedIdentifier));
+ }
+ return deleteValues;
+ }
+
+ @Override
+ public List<String> getSortKeys() {
+ return table.getStorageHandler().acidSortColumns(table, Context.Operation.DELETE).stream()
+ .map(fieldSchema -> String.format(
+ "%s.%s",
+ subQueryAlias,
+ HiveUtils.unparseIdentifier(DELETE_PREFIX + fieldSchema.getName(), this.conf)))
+ .collect(Collectors.toList());
+ }
+ }
}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java
index 1bfe69bc2d9..8bc219e7649 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java
@@ -35,8 +35,6 @@ import org.apache.hadoop.hive.ql.lib.Node;
import org.apache.hadoop.hive.ql.metadata.HiveUtils;
import org.apache.hadoop.hive.ql.metadata.Table;
-import static java.util.Collections.singletonList;
-
/**
* A subclass of the {@link org.apache.hadoop.hive.ql.parse.SemanticAnalyzer} that just handles
* update and delete statements. It works by rewriting the updates and deletes into insert
@@ -76,8 +74,6 @@ public class UpdateDeleteSemanticAnalyzer extends RewriteSemanticAnalyzer {
private void analyzeUpdate(ASTNode tree, Table mTable, ASTNode tabNameNode) throws SemanticException {
operation = Context.Operation.UPDATE;
- boolean nonNativeAcid = AcidUtils.isNonNativeAcidTable(mTable);
-
if (HiveConf.getBoolVar(queryState.getConf(), HiveConf.ConfVars.SPLIT_UPDATE)) {
analyzeSplitUpdate(tree, mTable, tabNameNode);
} else {
@@ -159,7 +155,7 @@ public class UpdateDeleteSemanticAnalyzer extends RewriteSemanticAnalyzer {
}
}
- addPartitionColsToSelect(mTable.getPartCols(), rewrittenQueryStr);
+ addColsToSelect(mTable.getPartCols(), rewrittenQueryStr);
rewrittenQueryStr.append(" from ");
rewrittenQueryStr.append(getFullTableNameForSQL(tabNameNode));
@@ -253,9 +249,6 @@ public class UpdateDeleteSemanticAnalyzer extends RewriteSemanticAnalyzer {
}
}
- public static final String DELETE_PREFIX = "__d__";
- public static final String SUB_QUERY_ALIAS = "s";
-
private void analyzeSplitUpdate(ASTNode tree, Table mTable, ASTNode tabNameNode) throws SemanticException {
operation = Context.Operation.UPDATE;
@@ -284,31 +277,10 @@ public class UpdateDeleteSemanticAnalyzer extends RewriteSemanticAnalyzer {
StringBuilder rewrittenQueryStr = createRewrittenQueryStrBuilder();
rewrittenQueryStr.append("(SELECT ");
- boolean nonNativeAcid = AcidUtils.isNonNativeAcidTable(mTable);
- int columnOffset;
- List<String> deleteValues;
- if (nonNativeAcid) {
- List<FieldSchema> acidSelectColumns = mTable.getStorageHandler().acidSelectColumns(mTable, operation);
- deleteValues = new ArrayList<>(acidSelectColumns.size());
- for (FieldSchema fieldSchema : acidSelectColumns) {
- String identifier = HiveUtils.unparseIdentifier(fieldSchema.getName(), this.conf);
- rewrittenQueryStr.append(identifier).append(" AS ");
- String prefixedIdentifier = HiveUtils.unparseIdentifier(DELETE_PREFIX + fieldSchema.getName(), this.conf);
- rewrittenQueryStr.append(prefixedIdentifier);
- rewrittenQueryStr.append(",");
- deleteValues.add(String.format("%s.%s", SUB_QUERY_ALIAS, prefixedIdentifier));
- }
-
- columnOffset = acidSelectColumns.size();
- } else {
- rewrittenQueryStr.append("ROW__ID,");
- deleteValues = new ArrayList<>(1 + mTable.getPartCols().size());
- deleteValues.add(SUB_QUERY_ALIAS + ".ROW__ID");
- for (FieldSchema fieldSchema : mTable.getPartCols()) {
- deleteValues.add(SUB_QUERY_ALIAS + "." + HiveUtils.unparseIdentifier(fieldSchema.getName(), conf));
- }
- columnOffset = 1;
- }
+ ColumnAppender columnAppender = getColumnAppender(SUB_QUERY_ALIAS);
+ columnAppender.appendAcidSelectColumns(rewrittenQueryStr, operation);
+ List<String> deleteValues = columnAppender.getDeleteValues(operation);
+ int columnOffset = deleteValues.size();
List<String> insertValues = new ArrayList<>(mTable.getCols().size());
boolean first = true;
@@ -342,7 +314,6 @@ public class UpdateDeleteSemanticAnalyzer extends RewriteSemanticAnalyzer {
insertValues.add(SUB_QUERY_ALIAS + "." + identifier);
}
- addPartitionColsToSelect(mTable.getPartCols(), rewrittenQueryStr);
addPartitionColsAsValues(mTable.getPartCols(), SUB_QUERY_ALIAS, insertValues);
rewrittenQueryStr.append(" FROM ").append(getFullTableNameForSQL(tabNameNode)).append(") ");
rewrittenQueryStr.append(SUB_QUERY_ALIAS).append("\n");
@@ -350,17 +321,7 @@ public class UpdateDeleteSemanticAnalyzer extends RewriteSemanticAnalyzer {
appendInsertBranch(rewrittenQueryStr, null, insertValues);
appendInsertBranch(rewrittenQueryStr, null, deleteValues);
- List<String> sortKeys;
- if (nonNativeAcid) {
- sortKeys = mTable.getStorageHandler().acidSortColumns(mTable, Context.Operation.DELETE).stream()
- .map(fieldSchema -> String.format(
- "%s.%s",
- SUB_QUERY_ALIAS,
- HiveUtils.unparseIdentifier(DELETE_PREFIX + fieldSchema.getName(), this.conf)))
- .collect(Collectors.toList());
- } else {
- sortKeys = singletonList(SUB_QUERY_ALIAS + ".ROW__ID ");
- }
+ List<String> sortKeys = columnAppender.getSortKeys();
appendSortBy(rewrittenQueryStr, sortKeys);
ReparseResult rr = parseRewrittenQuery(rewrittenQueryStr, ctx.getCmd());
diff --git a/ql/src/test/results/clientpositive/llap/acid_direct_update_delete_with_merge.q.out b/ql/src/test/results/clientpositive/llap/acid_direct_update_delete_with_merge.q.out
index c50b67dc4fc..ce703e40a78 100644
--- a/ql/src/test/results/clientpositive/llap/acid_direct_update_delete_with_merge.q.out
+++ b/ql/src/test/results/clientpositive/llap/acid_direct_update_delete_with_merge.q.out
@@ -113,7 +113,7 @@ POSTHOOK: Output: default@transactions@tran_date=20170410
POSTHOOK: Output: default@transactions@tran_date=20170413
POSTHOOK: Output: default@transactions@tran_date=20170413
POSTHOOK: Output: default@transactions@tran_date=20170415
-POSTHOOK: Lineage: merge_tmp_table.val EXPRESSION [(transactions)t.FieldSchema(name:ROW__ID, type:struct<writeId:bigint,bucketId:int,rowId:bigint>, comment:), (transactions)t.FieldSchema(name:tran_date, type:string, comment:null), ]
+POSTHOOK: Lineage: merge_tmp_table.val EXPRESSION [(transactions)transactions.FieldSchema(name:ROW__ID, type:struct<writeId:bigint,bucketId:int,rowId:bigint>, comment:), (transactions)transactions.FieldSchema(name:tran_date, type:string, comment:null), ]
POSTHOOK: Lineage: transactions PARTITION(tran_date=20170413).id SIMPLE [(merge_source)s.FieldSchema(name:id, type:int, comment:null), ]
POSTHOOK: Lineage: transactions PARTITION(tran_date=20170413).last_update_user SIMPLE []
POSTHOOK: Lineage: transactions PARTITION(tran_date=20170413).value SIMPLE [(merge_source)s.FieldSchema(name:value, type:string, comment:null), ]
diff --git a/ql/src/test/results/clientpositive/llap/acid_no_buckets.q.out b/ql/src/test/results/clientpositive/llap/acid_no_buckets.q.out
index 0a0483d52a2..5e812d19d01 100644
--- a/ql/src/test/results/clientpositive/llap/acid_no_buckets.q.out
+++ b/ql/src/test/results/clientpositive/llap/acid_no_buckets.q.out
@@ -148,11 +148,11 @@ STAGE PLANS:
predicate: (UDFToInteger(key)) IN (413, 43) (type: boolean)
Statistics: Num rows: 500 Data size: 181000 Basic stats: COMPLETE Column stats: PARTIAL
Select Operator
- expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), key (type: string), concat(value, 'updated') (type: string), ds (type: string)
- outputColumnNames: _col0, _col1, _col2, _col3
+ expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), ds (type: string), key (type: string), concat(value, 'updated') (type: string)
+ outputColumnNames: _col0, _col1, _col3, _col4
Statistics: Num rows: 500 Data size: 265500 Basic stats: COMPLETE Column stats: PARTIAL
Select Operator
- expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col3 (type: string)
+ expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col1 (type: string)
outputColumnNames: _col0, _col1
Statistics: Num rows: 500 Data size: 130000 Basic stats: COMPLETE Column stats: PARTIAL
Reduce Output Operator
@@ -163,7 +163,7 @@ STAGE PLANS:
Statistics: Num rows: 500 Data size: 130000 Basic stats: COMPLETE Column stats: PARTIAL
value expressions: _col1 (type: string)
Select Operator
- expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string), '11' (type: string)
+ expressions: _col3 (type: string), _col4 (type: string), _col1 (type: string), '11' (type: string)
outputColumnNames: _col0, _col1, _col2, _col3
Statistics: Num rows: 500 Data size: 270500 Basic stats: COMPLETE Column stats: PARTIAL
File Output Operator
@@ -655,7 +655,7 @@ POSTHOOK: Output: default@srcpart_acid@ds=2008-04-08/hr=11
POSTHOOK: Output: default@srcpart_acid@ds=2008-04-08/hr=12
POSTHOOK: Output: default@srcpart_acid@ds=2008-04-09/hr=11
POSTHOOK: Output: default@srcpart_acid@ds=2008-04-09/hr=12
-POSTHOOK: Lineage: merge_tmp_table.val EXPRESSION [(srcpart_acid)t.FieldSchema(name:ROW__ID, type:struct<writeId:bigint,bucketId:int,rowId:bigint>, comment:), (srcpart_acid)t.FieldSchema(name:ds, type:string, comment:null), (srcpart_acid)t.FieldSchema(name:hr, type:string, comment:null), ]
+POSTHOOK: Lineage: merge_tmp_table.val EXPRESSION [(srcpart_acid)srcpart_acid.FieldSchema(name:ROW__ID, type:struct<writeId:bigint,bucketId:int,rowId:bigint>, comment:), (srcpart_acid)srcpart_acid.FieldSchema(name:ds, type:string, comment:null), (srcpart_acid)srcpart_acid.FieldSchema(name:hr, type:string, comment:null), ]
POSTHOOK: Lineage: srcpart_acid PARTITION(ds=2008-04-08,hr=11).key SIMPLE []
POSTHOOK: Lineage: srcpart_acid PARTITION(ds=2008-04-08,hr=11).value SIMPLE []
PREHOOK: query: select count(*) from srcpart_acid where ds='2008-04-08' and hr=='12'
@@ -852,11 +852,11 @@ STAGE PLANS:
predicate: (UDFToInteger(key)) IN (413, 43) (type: boolean)
Statistics: Num rows: 500 Data size: 181000 Basic stats: COMPLETE Column stats: PARTIAL
Select Operator
- expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), key (type: string), concat(value, 'updated') (type: string), ds (type: string)
- outputColumnNames: _col0, _col1, _col2, _col3
+ expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), ds (type: string), key (type: string), concat(value, 'updated') (type: string)
+ outputColumnNames: _col0, _col1, _col3, _col4
Statistics: Num rows: 500 Data size: 265500 Basic stats: COMPLETE Column stats: PARTIAL
Select Operator
- expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col3 (type: string)
+ expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col1 (type: string)
outputColumnNames: _col0, _col1
Statistics: Num rows: 500 Data size: 130000 Basic stats: COMPLETE Column stats: PARTIAL
Reduce Output Operator
@@ -867,7 +867,7 @@ STAGE PLANS:
Statistics: Num rows: 500 Data size: 173000 Basic stats: COMPLETE Column stats: PARTIAL
value expressions: _col1 (type: string), '11' (type: string)
Select Operator
- expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string), '11' (type: string)
+ expressions: _col3 (type: string), _col4 (type: string), _col1 (type: string), '11' (type: string)
outputColumnNames: _col0, _col1, _col2, _col3
Statistics: Num rows: 500 Data size: 270500 Basic stats: COMPLETE Column stats: PARTIAL
Reduce Output Operator
@@ -1253,7 +1253,7 @@ POSTHOOK: Output: default@srcpart_acidb@ds=2008-04-08/hr=11
POSTHOOK: Output: default@srcpart_acidb@ds=2008-04-08/hr=12
POSTHOOK: Output: default@srcpart_acidb@ds=2008-04-09/hr=11
POSTHOOK: Output: default@srcpart_acidb@ds=2008-04-09/hr=12
-POSTHOOK: Lineage: merge_tmp_table.val EXPRESSION [(srcpart_acidb)t.FieldSchema(name:ROW__ID, type:struct<writeId:bigint,bucketId:int,rowId:bigint>, comment:), (srcpart_acidb)t.FieldSchema(name:ds, type:string, comment:null), (srcpart_acidb)t.FieldSchema(name:hr, type:string, comment:null), ]
+POSTHOOK: Lineage: merge_tmp_table.val EXPRESSION [(srcpart_acidb)srcpart_acidb.FieldSchema(name:ROW__ID, type:struct<writeId:bigint,bucketId:int,rowId:bigint>, comment:), (srcpart_acidb)srcpart_acidb.FieldSchema(name:ds, type:string, comment:null), (srcpart_acidb)srcpart_acidb.FieldSchema(name:hr, type:string, comment:null), ]
POSTHOOK: Lineage: srcpart_acidb PARTITION(ds=2008-04-08,hr=11).key SIMPLE []
POSTHOOK: Lineage: srcpart_acidb PARTITION(ds=2008-04-08,hr=11).value SIMPLE []
PREHOOK: query: select count(*) from srcpart_acidb where ds='2008-04-08' and hr=='12'
@@ -1455,7 +1455,7 @@ STAGE PLANS:
Select Vectorization:
className: VectorSelectOperator
native: true
- projectedOutputColumnNums: [4, 0, 7, 2]
+ projectedOutputColumnNums: [4, 2, 0, 7]
selectExpressions: StringGroupColConcatStringScalar(col 1:string, val updated) -> 7:string
Select Vectorization:
className: VectorSelectOperator
@@ -1886,10 +1886,10 @@ STAGE PLANS:
Select Vectorization:
className: VectorSelectOperator
native: true
- projectedOutputColumnNums: [0, 1, 2, 3, 4]
+ projectedOutputColumnNums: [4, 2, 3, 0, 1]
Reduce Sink Vectorization:
className: VectorReduceSinkMultiKeyOperator
- keyColumns: 0:string, 1:string, 2:string, 3:string
+ keyColumns: 2:string, 3:string, 0:string, 1:string
native: true
nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
valueColumns: 4:struct<writeid:bigint,bucketid:int,rowid:bigint>
@@ -1924,7 +1924,7 @@ STAGE PLANS:
Group By Vectorization:
className: VectorGroupByOperator
groupByMode: HASH
- keyExpressions: col 0:string, col 1:string, col 2:string, col 3:string
+ keyExpressions: col 2:string, col 3:string, col 0:string, col 1:string
native: false
vectorProcessingMode: HASH
projectedOutputColumnNums: []
@@ -2032,43 +2032,39 @@ STAGE PLANS:
native: false
vectorProcessingMode: MERGE_PARTIAL
projectedOutputColumnNums: []
+ Reduce Sink Vectorization:
+ className: VectorReduceSinkMultiKeyOperator
+ keyColumns: 0:string, 1:string, 2:string, 3:string
+ native: true
+ nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
Select Vectorization:
className: VectorSelectOperator
native: true
- projectedOutputColumnNums: [2, 3, 0, 1]
- Reduce Sink Vectorization:
- className: VectorReduceSinkMultiKeyOperator
- keyColumns: 0:string, 1:string, 2:string, 3:string
- native: true
- nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
- Select Vectorization:
- className: VectorSelectOperator
- native: true
- projectedOutputColumnNums: [2]
- Group By Vectorization:
- className: VectorGroupByOperator
- groupByMode: HASH
- keyExpressions: col 2:string
- native: false
- vectorProcessingMode: HASH
- projectedOutputColumnNums: []
- App Master Event Vectorization:
- className: VectorAppMasterEventOperator
- native: true
- Select Vectorization:
- className: VectorSelectOperator
- native: true
- projectedOutputColumnNums: [3]
- Group By Vectorization:
- className: VectorGroupByOperator
- groupByMode: HASH
- keyExpressions: col 3:string
- native: false
- vectorProcessingMode: HASH
- projectedOutputColumnNums: []
- App Master Event Vectorization:
- className: VectorAppMasterEventOperator
- native: true
+ projectedOutputColumnNums: [0]
+ Group By Vectorization:
+ className: VectorGroupByOperator
+ groupByMode: HASH
+ keyExpressions: col 0:string
+ native: false
+ vectorProcessingMode: HASH
+ projectedOutputColumnNums: []
+ App Master Event Vectorization:
+ className: VectorAppMasterEventOperator
+ native: true
+ Select Vectorization:
+ className: VectorSelectOperator
+ native: true
+ projectedOutputColumnNums: [1]
+ Group By Vectorization:
+ className: VectorGroupByOperator
+ groupByMode: HASH
+ keyExpressions: col 1:string
+ native: false
+ vectorProcessingMode: HASH
+ projectedOutputColumnNums: []
+ App Master Event Vectorization:
+ className: VectorAppMasterEventOperator
+ native: true
Stage: Stage-6
@@ -2127,7 +2123,7 @@ POSTHOOK: Output: default@srcpart_acidv@ds=2008-04-08/hr=11
POSTHOOK: Output: default@srcpart_acidv@ds=2008-04-08/hr=12
POSTHOOK: Output: default@srcpart_acidv@ds=2008-04-09/hr=11
POSTHOOK: Output: default@srcpart_acidv@ds=2008-04-09/hr=12
-POSTHOOK: Lineage: merge_tmp_table.val EXPRESSION [(srcpart_acidv)t.FieldSchema(name:ROW__ID, type:struct<writeId:bigint,bucketId:int,rowId:bigint>, comment:), (srcpart_acidv)t.FieldSchema(name:ds, type:string, comment:null), (srcpart_acidv)t.FieldSchema(name:hr, type:string, comment:null), ]
+POSTHOOK: Lineage: merge_tmp_table.val EXPRESSION [(srcpart_acidv)srcpart_acidv.FieldSchema(name:ROW__ID, type:struct<writeId:bigint,bucketId:int,rowId:bigint>, comment:), (srcpart_acidv)srcpart_acidv.FieldSchema(name:ds, type:string, comment:null), (srcpart_acidv)srcpart_acidv.FieldSchema(name:hr, type:string, comment:null), ]
POSTHOOK: Lineage: srcpart_acidv PARTITION(ds=2008-04-08,hr=11).key SIMPLE []
POSTHOOK: Lineage: srcpart_acidv PARTITION(ds=2008-04-08,hr=11).value SIMPLE []
PREHOOK: query: select count(*) from srcpart_acidv where ds='2008-04-08' and hr=='12'
@@ -2330,7 +2326,7 @@ STAGE PLANS:
Select Vectorization:
className: VectorSelectOperator
native: true
- projectedOutputColumnNums: [4, 0, 7, 2]
+ projectedOutputColumnNums: [4, 2, 0, 7]
selectExpressions: StringGroupColConcatStringScalar(col 1:string, val updated) -> 7:string
Select Vectorization:
className: VectorSelectOperator
@@ -2790,10 +2786,10 @@ STAGE PLANS:
Select Vectorization:
className: VectorSelectOperator
native: true
- projectedOutputColumnNums: [0, 1, 2, 3, 4]
+ projectedOutputColumnNums: [4, 2, 3, 0, 1]
Reduce Sink Vectorization:
className: VectorReduceSinkMultiKeyOperator
- keyColumns: 0:string, 1:string, 2:string, 3:string
+ keyColumns: 2:string, 3:string, 0:string, 1:string
native: true
nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
valueColumns: 4:struct<writeid:bigint,bucketid:int,rowid:bigint>
@@ -2828,7 +2824,7 @@ STAGE PLANS:
Group By Vectorization:
className: VectorGroupByOperator
groupByMode: HASH
- keyExpressions: col 0:string, col 1:string, col 2:string, col 3:string
+ keyExpressions: col 2:string, col 3:string, col 0:string, col 1:string
native: false
vectorProcessingMode: HASH
projectedOutputColumnNums: []
@@ -2983,43 +2979,39 @@ STAGE PLANS:
native: false
vectorProcessingMode: MERGE_PARTIAL
projectedOutputColumnNums: []
+ Reduce Sink Vectorization:
+ className: VectorReduceSinkMultiKeyOperator
+ keyColumns: 0:string, 1:string, 2:string, 3:string
+ native: true
+ nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
Select Vectorization:
className: VectorSelectOperator
native: true
- projectedOutputColumnNums: [2, 3, 0, 1]
- Reduce Sink Vectorization:
- className: VectorReduceSinkMultiKeyOperator
- keyColumns: 0:string, 1:string, 2:string, 3:string
- native: true
- nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
- Select Vectorization:
- className: VectorSelectOperator
- native: true
- projectedOutputColumnNums: [2]
- Group By Vectorization:
- className: VectorGroupByOperator
- groupByMode: HASH
- keyExpressions: col 2:string
- native: false
- vectorProcessingMode: HASH
- projectedOutputColumnNums: []
- App Master Event Vectorization:
- className: VectorAppMasterEventOperator
- native: true
- Select Vectorization:
- className: VectorSelectOperator
- native: true
- projectedOutputColumnNums: [3]
- Group By Vectorization:
- className: VectorGroupByOperator
- groupByMode: HASH
- keyExpressions: col 3:string
- native: false
- vectorProcessingMode: HASH
- projectedOutputColumnNums: []
- App Master Event Vectorization:
- className: VectorAppMasterEventOperator
- native: true
+ projectedOutputColumnNums: [0]
+ Group By Vectorization:
+ className: VectorGroupByOperator
+ groupByMode: HASH
+ keyExpressions: col 0:string
+ native: false
+ vectorProcessingMode: HASH
+ projectedOutputColumnNums: []
+ App Master Event Vectorization:
+ className: VectorAppMasterEventOperator
+ native: true
+ Select Vectorization:
+ className: VectorSelectOperator
+ native: true
+ projectedOutputColumnNums: [1]
+ Group By Vectorization:
+ className: VectorGroupByOperator
+ groupByMode: HASH
+ keyExpressions: col 1:string
+ native: false
+ vectorProcessingMode: HASH
+ projectedOutputColumnNums: []
+ App Master Event Vectorization:
+ className: VectorAppMasterEventOperator
+ native: true
Stage: Stage-6
@@ -3078,7 +3070,7 @@ POSTHOOK: Output: default@srcpart_acidvb@ds=2008-04-08/hr=11
POSTHOOK: Output: default@srcpart_acidvb@ds=2008-04-08/hr=12
POSTHOOK: Output: default@srcpart_acidvb@ds=2008-04-09/hr=11
POSTHOOK: Output: default@srcpart_acidvb@ds=2008-04-09/hr=12
-POSTHOOK: Lineage: merge_tmp_table.val EXPRESSION [(srcpart_acidvb)t.FieldSchema(name:ROW__ID, type:struct<writeId:bigint,bucketId:int,rowId:bigint>, comment:), (srcpart_acidvb)t.FieldSchema(name:ds, type:string, comment:null), (srcpart_acidvb)t.FieldSchema(name:hr, type:string, comment:null), ]
+POSTHOOK: Lineage: merge_tmp_table.val EXPRESSION [(srcpart_acidvb)srcpart_acidvb.FieldSchema(name:ROW__ID, type:struct<writeId:bigint,bucketId:int,rowId:bigint>, comment:), (srcpart_acidvb)srcpart_acidvb.FieldSchema(name:ds, type:string, comment:null), (srcpart_acidvb)srcpart_acidvb.FieldSchema(name:hr, type:string, comment:null), ]
POSTHOOK: Lineage: srcpart_acidvb PARTITION(ds=2008-04-08,hr=11).key SIMPLE []
POSTHOOK: Lineage: srcpart_acidvb PARTITION(ds=2008-04-08,hr=11).value SIMPLE []
PREHOOK: query: select count(*) from srcpart_acidvb where ds='2008-04-08' and hr=='12'
diff --git a/ql/src/test/results/clientpositive/llap/acid_subquery.q.out b/ql/src/test/results/clientpositive/llap/acid_subquery.q.out
index c62ee9a8b44..d970d62b9c0 100644
--- a/ql/src/test/results/clientpositive/llap/acid_subquery.q.out
+++ b/ql/src/test/results/clientpositive/llap/acid_subquery.q.out
@@ -249,7 +249,7 @@ POSTHOOK: Output: default@target@p=1/q=2
POSTHOOK: Output: default@target@p=1/q=3
POSTHOOK: Output: default@target@p=111/q=111
POSTHOOK: Output: default@target@p=2/q=2
-POSTHOOK: Lineage: merge_tmp_table.val EXPRESSION [(target)t.FieldSchema(name:ROW__ID, type:struct<writeId:bigint,bucketId:int,rowId:bigint>, comment:), (target)t.FieldSchema(name:p, type:int, comment:null), (target)t.FieldSchema(name:q, type:int, comment:null), ]
+POSTHOOK: Lineage: merge_tmp_table.val EXPRESSION [(target)target.FieldSchema(name:ROW__ID, type:struct<writeId:bigint,bucketId:int,rowId:bigint>, comment:), (target)target.FieldSchema(name:p, type:int, comment:null), (target)target.FieldSchema(name:q, type:int, comment:null), ]
POSTHOOK: Lineage: target PARTITION(p=1,q=2).a SIMPLE [(source)s.FieldSchema(name:a1, type:int, comment:null), ]
POSTHOOK: Lineage: target PARTITION(p=1,q=2).b SIMPLE [(source)s.FieldSchema(name:b1, type:int, comment:null), ]
POSTHOOK: Lineage: target PARTITION(p=111,q=111).a SIMPLE [(source)s.FieldSchema(name:a1, type:int, comment:null), ]
diff --git a/ql/src/test/results/clientpositive/llap/check_constraint.q.out b/ql/src/test/results/clientpositive/llap/check_constraint.q.out
index a63867310e8..b0127cc834c 100644
--- a/ql/src/test/results/clientpositive/llap/check_constraint.q.out
+++ b/ql/src/test/results/clientpositive/llap/check_constraint.q.out
@@ -2519,23 +2519,23 @@ STAGE PLANS:
Map 7
Map Operator Tree:
TableScan
- alias: t
+ alias: tmerge
filterExpr: key is not null (type: boolean)
Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
Filter Operator
predicate: key is not null (type: boolean)
Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: key (type: int), value (type: string), ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), key (type: int), value (type: string)
outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
Reduce Output Operator
- key expressions: _col0 (type: int)
+ key expressions: _col1 (type: int)
null sort order: z
sort order: +
- Map-reduce partition columns: _col0 (type: int)
+ Map-reduce partition columns: _col1 (type: int)
Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col1 (type: string), _col2 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ value expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col2 (type: string)
Execution mode: vectorized, llap
LLAP IO: may be used (ACID table)
Reducer 2
@@ -2546,11 +2546,11 @@ STAGE PLANS:
Left Outer Join 0 to 1
keys:
0 _col0 (type: int)
- 1 _col0 (type: int)
+ 1 _col1 (type: int)
outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
Statistics: Num rows: 1 Data size: 409 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: _col1 (type: string), _col0 (type: int), _col4 (type: string), _col5 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col2 (type: string), _col3 (type: int)
+ expressions: _col1 (type: string), _col0 (type: int), _col5 (type: string), _col3 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col2 (type: string), _col4 (type: int)
outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
Statistics: Num rows: 1 Data size: 409 Basic stats: COMPLETE Column stats: NONE
Filter Operator
@@ -2788,23 +2788,23 @@ STAGE PLANS:
Map 8
Map Operator Tree:
TableScan
- alias: t
+ alias: tmerge
filterExpr: key is not null (type: boolean)
Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
Filter Operator
predicate: key is not null (type: boolean)
Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: key (type: int), value (type: string), ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), key (type: int), value (type: string)
outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
Reduce Output Operator
- key expressions: _col0 (type: int)
+ key expressions: _col1 (type: int)
null sort order: z
sort order: +
- Map-reduce partition columns: _col0 (type: int)
+ Map-reduce partition columns: _col1 (type: int)
Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col1 (type: string), _col2 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ value expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col2 (type: string)
Execution mode: vectorized, llap
LLAP IO: may be used (ACID table)
Reducer 2
@@ -2815,11 +2815,11 @@ STAGE PLANS:
Left Outer Join 0 to 1
keys:
0 _col0 (type: int)
- 1 _col0 (type: int)
+ 1 _col1 (type: int)
outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
Statistics: Num rows: 1 Data size: 409 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: _col1 (type: string), _col0 (type: int), _col4 (type: string), _col5 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col2 (type: string), _col3 (type: int)
+ expressions: _col1 (type: string), _col0 (type: int), _col5 (type: string), _col3 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col2 (type: string), _col4 (type: int)
outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
Statistics: Num rows: 1 Data size: 409 Basic stats: COMPLETE Column stats: NONE
Filter Operator
@@ -3106,23 +3106,23 @@ STAGE PLANS:
Map 6
Map Operator Tree:
TableScan
- alias: t
+ alias: tmerge
filterExpr: key is not null (type: boolean)
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Filter Operator
predicate: key is not null (type: boolean)
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: key (type: int), ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), key (type: int)
outputColumnNames: _col0, _col1
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Reduce Output Operator
- key expressions: _col0 (type: int)
+ key expressions: _col1 (type: int)
null sort order: z
sort order: +
- Map-reduce partition columns: _col0 (type: int)
+ Map-reduce partition columns: _col1 (type: int)
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col1 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ value expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
Execution mode: vectorized, llap
LLAP IO: may be used (ACID table)
Reducer 2
@@ -3133,11 +3133,11 @@ STAGE PLANS:
Left Outer Join 0 to 1
keys:
0 _col0 (type: int)
- 1 _col0 (type: int)
+ 1 _col1 (type: int)
outputColumnNames: _col0, _col1, _col2, _col3, _col4
Statistics: Num rows: 1 Data size: 409 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: _col1 (type: string), _col0 (type: int), _col4 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col2 (type: string), _col3 (type: int)
+ expressions: _col1 (type: string), _col0 (type: int), _col3 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col2 (type: string), _col4 (type: int)
outputColumnNames: _col0, _col1, _col2, _col3, _col4
Statistics: Num rows: 1 Data size: 409 Basic stats: COMPLETE Column stats: NONE
Filter Operator
diff --git a/ql/src/test/results/clientpositive/llap/create_transactional_full_acid.q.out b/ql/src/test/results/clientpositive/llap/create_transactional_full_acid.q.out
index 77673d15e55..a3421aff8e6 100644
--- a/ql/src/test/results/clientpositive/llap/create_transactional_full_acid.q.out
+++ b/ql/src/test/results/clientpositive/llap/create_transactional_full_acid.q.out
@@ -197,4 +197,4 @@ POSTHOOK: Output: default@target
POSTHOOK: Output: default@target@p=1/q=2
POSTHOOK: Output: default@target@p=1/q=3
POSTHOOK: Output: default@target@p=2/q=2
-POSTHOOK: Lineage: merge_tmp_table.val EXPRESSION [(target)t.FieldSchema(name:ROW__ID, type:struct<writeId:bigint,bucketId:int,rowId:bigint>, comment:), (target)t.FieldSchema(name:p, type:int, comment:null), (target)t.FieldSchema(name:q, type:int, comment:null), ]
+POSTHOOK: Lineage: merge_tmp_table.val EXPRESSION [(target)target.FieldSchema(name:ROW__ID, type:struct<writeId:bigint,bucketId:int,rowId:bigint>, comment:), (target)target.FieldSchema(name:p, type:int, comment:null), (target)target.FieldSchema(name:q, type:int, comment:null), ]
diff --git a/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_3.q.out b/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_3.q.out
index 0c64fc28f45..bc28b56a46d 100644
--- a/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_3.q.out
+++ b/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_3.q.out
@@ -99,23 +99,23 @@ STAGE PLANS:
Map 9
Map Operator Tree:
TableScan
- alias: t
+ alias: acidtbl
filterExpr: (a is not null and a BETWEEN DynamicValue(RS_5_s_a_min) AND DynamicValue(RS_5_s_a_max) and in_bloom_filter(a, DynamicValue(RS_5_s_a_bloom_filter))) (type: boolean)
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Filter Operator
predicate: (a is not null and a BETWEEN DynamicValue(RS_5_s_a_min) AND DynamicValue(RS_5_s_a_max) and in_bloom_filter(a, DynamicValue(RS_5_s_a_bloom_filter))) (type: boolean)
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: a (type: int), ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), a (type: int)
outputColumnNames: _col0, _col1
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Reduce Output Operator
- key expressions: _col0 (type: int)
+ key expressions: _col1 (type: int)
null sort order: z
sort order: +
- Map-reduce partition columns: _col0 (type: int)
+ Map-reduce partition columns: _col1 (type: int)
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col1 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ value expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
Execution mode: vectorized, llap
LLAP IO: may be used (ACID table)
Reducer 2
@@ -126,11 +126,11 @@ STAGE PLANS:
Left Outer Join 0 to 1
keys:
0 _col0 (type: int)
- 1 _col0 (type: int)
+ 1 _col1 (type: int)
outputColumnNames: _col0, _col1, _col2, _col3
Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: _col3 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col1 (type: int), _col2 (type: int), _col0 (type: int)
+ expressions: _col2 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col1 (type: int), _col3 (type: int), _col0 (type: int)
outputColumnNames: _col0, _col1, _col2, _col3
Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
Filter Operator
@@ -419,7 +419,7 @@ STAGE PLANS:
Map 1
Map Operator Tree:
TableScan
- alias: t
+ alias: acidtbl
filterExpr: (a is not null and a BETWEEN DynamicValue(RS_6_s_a_min) AND DynamicValue(RS_6_s_a_max) and in_bloom_filter(a, DynamicValue(RS_6_s_a_bloom_filter))) (type: boolean)
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Filter Operator
@@ -663,23 +663,23 @@ STAGE PLANS:
Map 10
Map Operator Tree:
TableScan
- alias: t
+ alias: acidtbl
filterExpr: a is not null (type: boolean)
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Filter Operator
predicate: a is not null (type: boolean)
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: a (type: int), ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), a (type: int)
outputColumnNames: _col0, _col1
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Reduce Output Operator
- key expressions: _col0 (type: int)
+ key expressions: _col1 (type: int)
null sort order: z
sort order: +
- Map-reduce partition columns: _col0 (type: int)
+ Map-reduce partition columns: _col1 (type: int)
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col1 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ value expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
Execution mode: vectorized, llap
LLAP IO: may be used (ACID table)
Map 9
@@ -712,11 +712,11 @@ STAGE PLANS:
Left Outer Join 0 to 1
keys:
0 _col0 (type: int)
- 1 _col0 (type: int)
+ 1 _col1 (type: int)
outputColumnNames: _col0, _col1, _col2, _col3
Statistics: Num rows: 2 Data size: 17 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: _col3 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col1 (type: int), _col2 (type: int), _col0 (type: int)
+ expressions: _col2 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col1 (type: int), _col3 (type: int), _col0 (type: int)
outputColumnNames: _col0, _col1, _col2, _col3
Statistics: Num rows: 2 Data size: 17 Basic stats: COMPLETE Column stats: NONE
Filter Operator
diff --git a/ql/src/test/results/clientpositive/llap/dynpart_sort_optimization_acid.q.out b/ql/src/test/results/clientpositive/llap/dynpart_sort_optimization_acid.q.out
index 90da25badfc..aed7dd71bd3 100644
--- a/ql/src/test/results/clientpositive/llap/dynpart_sort_optimization_acid.q.out
+++ b/ql/src/test/results/clientpositive/llap/dynpart_sort_optimization_acid.q.out
@@ -1110,21 +1110,17 @@ STAGE PLANS:
Statistics: Num rows: 10 Data size: 910 Basic stats: COMPLETE Column stats: PARTIAL
Select Operator
expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), hr (type: int)
- outputColumnNames: _col0, _col4
+ outputColumnNames: _col0, _col2
Statistics: Num rows: 10 Data size: 800 Basic stats: COMPLETE Column stats: PARTIAL
+ Reduce Output Operator
+ key expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: UDFToInteger(_col0) (type: int)
+ Statistics: Num rows: 10 Data size: 1740 Basic stats: COMPLETE Column stats: PARTIAL
+ value expressions: '2008-04-08' (type: string), _col2 (type: int)
Select Operator
- expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col4 (type: int)
- outputColumnNames: _col0, _col2
- Statistics: Num rows: 10 Data size: 800 Basic stats: COMPLETE Column stats: PARTIAL
- Reduce Output Operator
- key expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
- null sort order: z
- sort order: +
- Map-reduce partition columns: UDFToInteger(_col0) (type: int)
- Statistics: Num rows: 10 Data size: 1740 Basic stats: COMPLETE Column stats: PARTIAL
- value expressions: '2008-04-08' (type: string), _col2 (type: int)
- Select Operator
- expressions: 'foo' (type: string), 'bar' (type: string), '2008-04-08' (type: string), _col4 (type: int)
+ expressions: 'foo' (type: string), 'bar' (type: string), '2008-04-08' (type: string), _col2 (type: int)
outputColumnNames: _col0, _col1, _col2, _col3
Statistics: Num rows: 10 Data size: 2720 Basic stats: COMPLETE Column stats: PARTIAL
Reduce Output Operator
@@ -1686,20 +1682,16 @@ STAGE PLANS:
Statistics: Num rows: 10 Data size: 910 Basic stats: COMPLETE Column stats: PARTIAL
Select Operator
expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), hr (type: int)
- outputColumnNames: _col0, _col4
+ outputColumnNames: _col0, _col2
Statistics: Num rows: 10 Data size: 800 Basic stats: COMPLETE Column stats: PARTIAL
+ Reduce Output Operator
+ key expressions: '2008-04-08' (type: string), _col2 (type: int), _bucket_number (type: string), _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ null sort order: aaaa
+ sort order: ++++
+ Map-reduce partition columns: '2008-04-08' (type: string), _col2 (type: int)
+ Statistics: Num rows: 10 Data size: 1740 Basic stats: COMPLETE Column stats: PARTIAL
Select Operator
- expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col4 (type: int)
- outputColumnNames: _col0, _col2
- Statistics: Num rows: 10 Data size: 800 Basic stats: COMPLETE Column stats: PARTIAL
- Reduce Output Operator
- key expressions: '2008-04-08' (type: string), _col2 (type: int), _bucket_number (type: string), _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
- null sort order: aaaa
- sort order: ++++
- Map-reduce partition columns: '2008-04-08' (type: string), _col2 (type: int)
- Statistics: Num rows: 10 Data size: 1740 Basic stats: COMPLETE Column stats: PARTIAL
- Select Operator
- expressions: 'foo' (type: string), 'bar' (type: string), '2008-04-08' (type: string), _col4 (type: int)
+ expressions: 'foo' (type: string), 'bar' (type: string), '2008-04-08' (type: string), _col2 (type: int)
outputColumnNames: _col0, _col1, _col2, _col3
Statistics: Num rows: 10 Data size: 2720 Basic stats: COMPLETE Column stats: PARTIAL
Reduce Output Operator
@@ -2260,20 +2252,16 @@ STAGE PLANS:
Statistics: Num rows: 10 Data size: 910 Basic stats: COMPLETE Column stats: PARTIAL
Select Operator
expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), hr (type: int)
- outputColumnNames: _col0, _col4
+ outputColumnNames: _col0, _col2
Statistics: Num rows: 10 Data size: 800 Basic stats: COMPLETE Column stats: PARTIAL
+ Reduce Output Operator
+ key expressions: '2008-04-08' (type: string), _col2 (type: int), _bucket_number (type: string), _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ null sort order: aaaa
+ sort order: ++++
+ Map-reduce partition columns: '2008-04-08' (type: string), _col2 (type: int)
+ Statistics: Num rows: 10 Data size: 1740 Basic stats: COMPLETE Column stats: PARTIAL
Select Operator
- expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col4 (type: int)
- outputColumnNames: _col0, _col2
- Statistics: Num rows: 10 Data size: 800 Basic stats: COMPLETE Column stats: PARTIAL
- Reduce Output Operator
- key expressions: '2008-04-08' (type: string), _col2 (type: int), _bucket_number (type: string), _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
- null sort order: aaaa
- sort order: ++++
- Map-reduce partition columns: '2008-04-08' (type: string), _col2 (type: int)
- Statistics: Num rows: 10 Data size: 1740 Basic stats: COMPLETE Column stats: PARTIAL
- Select Operator
- expressions: 'foo' (type: string), 'bar' (type: string), '2008-04-08' (type: string), _col4 (type: int)
+ expressions: 'foo' (type: string), 'bar' (type: string), '2008-04-08' (type: string), _col2 (type: int)
outputColumnNames: _col0, _col1, _col2, _col3
Statistics: Num rows: 10 Data size: 2720 Basic stats: COMPLETE Column stats: PARTIAL
Reduce Output Operator
diff --git a/ql/src/test/results/clientpositive/llap/enforce_constraint_notnull.q.out b/ql/src/test/results/clientpositive/llap/enforce_constraint_notnull.q.out
index 0f4387894d2..324b2d7b010 100644
--- a/ql/src/test/results/clientpositive/llap/enforce_constraint_notnull.q.out
+++ b/ql/src/test/results/clientpositive/llap/enforce_constraint_notnull.q.out
@@ -4577,19 +4577,19 @@ STAGE PLANS:
Map 7
Map Operator Tree:
TableScan
- alias: t
+ alias: masking_test_n4
Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: key (type: int), value (type: string), ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), key (type: int), value (type: string)
outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
Reduce Output Operator
- key expressions: _col0 (type: int)
+ key expressions: _col1 (type: int)
null sort order: z
sort order: +
- Map-reduce partition columns: _col0 (type: int)
+ Map-reduce partition columns: _col1 (type: int)
Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col1 (type: string), _col2 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ value expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col2 (type: string)
Execution mode: vectorized, llap
LLAP IO: may be used (ACID table)
Reducer 2
@@ -4600,11 +4600,11 @@ STAGE PLANS:
Left Outer Join 0 to 1
keys:
0 _col0 (type: int)
- 1 _col0 (type: int)
+ 1 _col1 (type: int)
outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
Statistics: Num rows: 1 Data size: 409 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: _col1 (type: string), _col0 (type: int), _col4 (type: string), _col5 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col2 (type: string), _col3 (type: int)
+ expressions: _col1 (type: string), _col0 (type: int), _col5 (type: string), _col3 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col2 (type: string), _col4 (type: int)
outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
Statistics: Num rows: 1 Data size: 409 Basic stats: COMPLETE Column stats: NONE
Filter Operator
@@ -4863,19 +4863,19 @@ STAGE PLANS:
Map 8
Map Operator Tree:
TableScan
- alias: t
+ alias: masking_test_n4
Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: key (type: int), value (type: string), ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), key (type: int), value (type: string)
outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
Reduce Output Operator
- key expressions: _col0 (type: int)
+ key expressions: _col1 (type: int)
null sort order: z
sort order: +
- Map-reduce partition columns: _col0 (type: int)
+ Map-reduce partition columns: _col1 (type: int)
Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col1 (type: string), _col2 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ value expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col2 (type: string)
Execution mode: vectorized, llap
LLAP IO: may be used (ACID table)
Reducer 2
@@ -4886,11 +4886,11 @@ STAGE PLANS:
Left Outer Join 0 to 1
keys:
0 _col0 (type: int)
- 1 _col0 (type: int)
+ 1 _col1 (type: int)
outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
Statistics: Num rows: 1 Data size: 409 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: _col1 (type: string), _col0 (type: int), _col4 (type: string), _col5 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col2 (type: string), _col3 (type: int)
+ expressions: _col1 (type: string), _col0 (type: int), _col5 (type: string), _col3 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col2 (type: string), _col4 (type: int)
outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
Statistics: Num rows: 1 Data size: 409 Basic stats: COMPLETE Column stats: NONE
Filter Operator
@@ -5200,19 +5200,19 @@ STAGE PLANS:
Map 6
Map Operator Tree:
TableScan
- alias: t
+ alias: masking_test_n4
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: key (type: int), ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), key (type: int)
outputColumnNames: _col0, _col1
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Reduce Output Operator
- key expressions: _col0 (type: int)
+ key expressions: _col1 (type: int)
null sort order: z
sort order: +
- Map-reduce partition columns: _col0 (type: int)
+ Map-reduce partition columns: _col1 (type: int)
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col1 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ value expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
Execution mode: vectorized, llap
LLAP IO: may be used (ACID table)
Reducer 2
@@ -5223,11 +5223,11 @@ STAGE PLANS:
Left Outer Join 0 to 1
keys:
0 _col0 (type: int)
- 1 _col0 (type: int)
+ 1 _col1 (type: int)
outputColumnNames: _col0, _col1, _col2, _col3, _col4
Statistics: Num rows: 1 Data size: 409 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: _col1 (type: string), _col0 (type: int), _col4 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col2 (type: string), _col3 (type: int)
+ expressions: _col1 (type: string), _col0 (type: int), _col3 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col2 (type: string), _col4 (type: int)
outputColumnNames: _col0, _col1, _col2, _col3, _col4
Statistics: Num rows: 1 Data size: 409 Basic stats: COMPLETE Column stats: NONE
Filter Operator
@@ -5448,19 +5448,19 @@ STAGE PLANS:
Map 7
Map Operator Tree:
TableScan
- alias: t
+ alias: masking_test_n4
Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: key (type: int), value (type: string), ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), key (type: int), value (type: string)
outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
Reduce Output Operator
- key expressions: _col0 (type: int)
+ key expressions: _col1 (type: int)
null sort order: z
sort order: +
- Map-reduce partition columns: _col0 (type: int)
+ Map-reduce partition columns: _col1 (type: int)
Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col1 (type: string), _col2 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ value expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col2 (type: string)
Execution mode: vectorized, llap
LLAP IO: may be used (ACID table)
Reducer 2
@@ -5471,11 +5471,11 @@ STAGE PLANS:
Left Outer Join 0 to 1
keys:
0 _col0 (type: int)
- 1 _col0 (type: int)
+ 1 _col1 (type: int)
outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
Statistics: Num rows: 1 Data size: 409 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: _col1 (type: string), _col0 (type: int), _col4 (type: string), _col5 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col2 (type: string), _col3 (type: int)
+ expressions: _col1 (type: string), _col0 (type: int), _col5 (type: string), _col3 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col2 (type: string), _col4 (type: int)
outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
Statistics: Num rows: 1 Data size: 409 Basic stats: COMPLETE Column stats: NONE
Filter Operator
@@ -5714,24 +5714,6 @@ STAGE PLANS:
#### A masked pattern was here ####
Vertices:
Map 1
- Map Operator Tree:
- TableScan
- alias: t
- Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
- Select Operator
- expressions: key (type: int), ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
- outputColumnNames: _col0, _col1
- Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
- Reduce Output Operator
- key expressions: _col0 (type: int)
- null sort order: z
- sort order: +
- Map-reduce partition columns: _col0 (type: int)
- Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col1 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
- Execution mode: vectorized, llap
- LLAP IO: may be used (ACID table)
- Map 5
Map Operator Tree:
TableScan
alias: s
@@ -5752,6 +5734,24 @@ STAGE PLANS:
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Execution mode: vectorized, llap
LLAP IO: all inputs
+ Map 5
+ Map Operator Tree:
+ TableScan
+ alias: masking_test_n4
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), key (type: int)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col1 (type: int)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col1 (type: int)
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ Execution mode: vectorized, llap
+ LLAP IO: may be used (ACID table)
Reducer 2
Execution mode: llap
Reduce Operator Tree:
@@ -5760,47 +5760,43 @@ STAGE PLANS:
Inner Join 0 to 1
keys:
0 _col0 (type: int)
- 1 _col0 (type: int)
+ 1 _col1 (type: int)
outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
- Select Operator
- expressions: _col2 (type: int), _col1 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col0 (type: int)
- outputColumnNames: _col0, _col1, _col2
+ Filter Operator
+ predicate: ((_col2 = _col0) and (_col0 < 5)) (type: boolean)
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
- Filter Operator
- predicate: ((_col2 = _col0) and (_col0 < 5)) (type: boolean)
+ Select Operator
+ expressions: _col1 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ outputColumnNames: _col0
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
- Select Operator
- expressions: _col1 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
- outputColumnNames: _col0
+ Reduce Output Operator
+ key expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: UDFToInteger(_col0) (type: int)
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (_col2 = _col0) (type: boolean)
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col1 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ outputColumnNames: _col1
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+ Group By Operator
+ aggregations: count()
+ keys: _col1 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ minReductionHashAggr: 0.99
+ mode: hash
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Reduce Output Operator
key expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
null sort order: z
sort order: +
- Map-reduce partition columns: UDFToInteger(_col0) (type: int)
+ Map-reduce partition columns: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
- Filter Operator
- predicate: (_col2 = _col0) (type: boolean)
- Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
- Select Operator
- expressions: _col1 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
- outputColumnNames: _col1
- Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
- Group By Operator
- aggregations: count()
- keys: _col1 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
- minReductionHashAggr: 0.99
- mode: hash
- outputColumnNames: _col0, _col1
- Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
- Reduce Output Operator
- key expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
- null sort order: z
- sort order: +
- Map-reduce partition columns: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
- Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col1 (type: bigint)
+ value expressions: _col1 (type: bigint)
Reducer 3
Execution mode: vectorized, llap
Reduce Operator Tree:
diff --git a/ql/src/test/results/clientpositive/llap/insert_into_default_keyword.q.out b/ql/src/test/results/clientpositive/llap/insert_into_default_keyword.q.out
index a57f6e72b3b..65f02649099 100644
--- a/ql/src/test/results/clientpositive/llap/insert_into_default_keyword.q.out
+++ b/ql/src/test/results/clientpositive/llap/insert_into_default_keyword.q.out
@@ -2814,7 +2814,7 @@ STAGE PLANS:
Map 1
Map Operator Tree:
TableScan
- alias: t
+ alias: acidtable
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
Select Operator
expressions: key (type: int)
@@ -3056,19 +3056,19 @@ STAGE PLANS:
Map 8
Map Operator Tree:
TableScan
- alias: t
+ alias: acidtable
Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: COMPLETE
Select Operator
- expressions: key (type: int), value (type: string), ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), key (type: int), value (type: string)
outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 1 Data size: 170 Basic stats: COMPLETE Column stats: COMPLETE
Reduce Output Operator
- key expressions: _col0 (type: int)
+ key expressions: _col1 (type: int)
null sort order: z
sort order: +
- Map-reduce partition columns: _col0 (type: int)
+ Map-reduce partition columns: _col1 (type: int)
Statistics: Num rows: 1 Data size: 170 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: _col1 (type: string), _col2 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ value expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col2 (type: string)
Execution mode: vectorized, llap
LLAP IO: may be used (ACID table)
Reducer 2
@@ -3079,11 +3079,11 @@ STAGE PLANS:
Left Outer Join 0 to 1
keys:
0 _col0 (type: int)
- 1 _col0 (type: int)
+ 1 _col1 (type: int)
outputColumnNames: _col0, _col1, _col2, _col3, _col4
Statistics: Num rows: 2 Data size: 432 Basic stats: COMPLETE Column stats: COMPLETE
Select Operator
- expressions: _col1 (type: string), _col0 (type: int), _col3 (type: string), _col4 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col2 (type: int)
+ expressions: _col1 (type: string), _col0 (type: int), _col4 (type: string), _col2 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col3 (type: int)
outputColumnNames: _col0, _col1, _col2, _col3, _col4
Statistics: Num rows: 2 Data size: 432 Basic stats: COMPLETE Column stats: COMPLETE
Filter Operator
@@ -3359,7 +3359,7 @@ POSTHOOK: Lineage: acidtable.key SIMPLE [(nonacid_n1)s.FieldSchema(name:key, typ
POSTHOOK: Lineage: acidtable.key SIMPLE [(nonacid_n1)s.FieldSchema(name:key, type:int, comment:null), ]
POSTHOOK: Lineage: acidtable.value EXPRESSION []
POSTHOOK: Lineage: acidtable.value EXPRESSION []
-POSTHOOK: Lineage: merge_tmp_table.val EXPRESSION [(acidtable)t.FieldSchema(name:ROW__ID, type:struct<writeId:bigint,bucketId:int,rowId:bigint>, comment:), ]
+POSTHOOK: Lineage: merge_tmp_table.val EXPRESSION [(acidtable)acidtable.FieldSchema(name:ROW__ID, type:struct<writeId:bigint,bucketId:int,rowId:bigint>, comment:), ]
PREHOOK: query: select * from acidTable
PREHOOK: type: QUERY
PREHOOK: Input: default@acidtable
@@ -3416,7 +3416,7 @@ STAGE PLANS:
Map 1
Map Operator Tree:
TableScan
- alias: t
+ alias: acidtable2
filterExpr: key is not null (type: boolean)
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Filter Operator
diff --git a/ql/src/test/results/clientpositive/llap/llap_acid.q.out b/ql/src/test/results/clientpositive/llap/llap_acid.q.out
index 6530a8f05b9..7f6c9a7a133 100644
--- a/ql/src/test/results/clientpositive/llap/llap_acid.q.out
+++ b/ql/src/test/results/clientpositive/llap/llap_acid.q.out
@@ -285,7 +285,7 @@ STAGE PLANS:
Select Vectorization:
className: VectorSelectOperator
native: true
- projectedOutputColumnNums: [5, 2, 3, 4]
+ projectedOutputColumnNums: [5, 4, 2, 3]
Select Vectorization:
className: VectorSelectOperator
native: true
diff --git a/ql/src/test/results/clientpositive/llap/llap_acid_fast.q.out b/ql/src/test/results/clientpositive/llap/llap_acid_fast.q.out
index 8f6f948e940..a82f19a8bb9 100644
--- a/ql/src/test/results/clientpositive/llap/llap_acid_fast.q.out
+++ b/ql/src/test/results/clientpositive/llap/llap_acid_fast.q.out
@@ -279,7 +279,7 @@ STAGE PLANS:
Select Vectorization:
className: VectorSelectOperator
native: true
- projectedOutputColumnNums: [5, 2, 3, 4]
+ projectedOutputColumnNums: [5, 4, 2, 3]
Select Vectorization:
className: VectorSelectOperator
native: true
diff --git a/ql/src/test/results/clientpositive/llap/masking_acid_no_masking.q.out b/ql/src/test/results/clientpositive/llap/masking_acid_no_masking.q.out
index 7ce45a64f60..cb13912cdaf 100644
--- a/ql/src/test/results/clientpositive/llap/masking_acid_no_masking.q.out
+++ b/ql/src/test/results/clientpositive/llap/masking_acid_no_masking.q.out
@@ -62,4 +62,4 @@ POSTHOOK: Lineage: masking_acid_no_masking.key SIMPLE [(nonacid_n0)s.FieldSchema
POSTHOOK: Lineage: masking_acid_no_masking.key SIMPLE [(nonacid_n0)s.FieldSchema(name:key, type:int, comment:null), ]
POSTHOOK: Lineage: masking_acid_no_masking.value SIMPLE [(nonacid_n0)s.FieldSchema(name:value, type:string, comment:null), ]
POSTHOOK: Lineage: masking_acid_no_masking.value SIMPLE [(nonacid_n0)s.FieldSchema(name:value, type:string, comment:null), ]
-POSTHOOK: Lineage: merge_tmp_table.val EXPRESSION [(masking_acid_no_masking)t.FieldSchema(name:ROW__ID, type:struct<writeId:bigint,bucketId:int,rowId:bigint>, comment:), ]
+POSTHOOK: Lineage: merge_tmp_table.val EXPRESSION [(masking_acid_no_masking)masking_acid_no_masking.FieldSchema(name:ROW__ID, type:struct<writeId:bigint,bucketId:int,rowId:bigint>, comment:), ]
diff --git a/ql/src/test/results/clientpositive/llap/runtime_stats_merge.q.out b/ql/src/test/results/clientpositive/llap/runtime_stats_merge.q.out
index 08c69b11c32..8e4d33895f5 100644
--- a/ql/src/test/results/clientpositive/llap/runtime_stats_merge.q.out
+++ b/ql/src/test/results/clientpositive/llap/runtime_stats_merge.q.out
@@ -135,7 +135,7 @@ Stage-4
Select Operator [SEL_9] (runtime: rows=1 width=84)
Output:["_col0","_col1","_col2"]
Map Join Operator [MAPJOIN_40] (runtime: rows=1 width=84)
- Conds:SEL_2._col0=RS_44._col0(Inner),Output:["_col0","_col1","_col2"]
+ Conds:SEL_2._col1=RS_44._col0(Inner),Output:["_col0","_col1","_col2"]
<-Map 4 [BROADCAST_EDGE] vectorized, llap
BROADCAST [RS_44]
PartitionCols:_col0
diff --git a/ql/src/test/results/clientpositive/llap/semijoin_hint.q.out b/ql/src/test/results/clientpositive/llap/semijoin_hint.q.out
index 93264fa09f7..544e04deebe 100644
--- a/ql/src/test/results/clientpositive/llap/semijoin_hint.q.out
+++ b/ql/src/test/results/clientpositive/llap/semijoin_hint.q.out
@@ -3373,19 +3373,23 @@ STAGE PLANS:
Map 1
Map Operator Tree:
TableScan
- alias: t
- filterExpr: (a BETWEEN DynamicValue(RS_3_s_a_min) AND DynamicValue(RS_3_s_a_max) and in_bloom_filter(a, DynamicValue(RS_3_s_a_bloom_filter))) (type: boolean)
+ alias: acidtbl
+ filterExpr: (a BETWEEN DynamicValue(RS_4_s_a_min) AND DynamicValue(RS_4_s_a_max) and in_bloom_filter(a, DynamicValue(RS_4_s_a_bloom_filter))) (type: boolean)
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Filter Operator
- predicate: (a BETWEEN DynamicValue(RS_3_s_a_min) AND DynamicValue(RS_3_s_a_max) and in_bloom_filter(a, DynamicValue(RS_3_s_a_bloom_filter))) (type: boolean)
+ predicate: (a BETWEEN DynamicValue(RS_4_s_a_min) AND DynamicValue(RS_4_s_a_max) and in_bloom_filter(a, DynamicValue(RS_4_s_a_bloom_filter))) (type: boolean)
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
- Reduce Output Operator
- key expressions: a (type: int)
- null sort order: z
- sort order: +
- Map-reduce partition columns: a (type: int)
+ Select Operator
+ expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), a (type: int)
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
- value expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ Reduce Output Operator
+ key expressions: _col1 (type: int)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col1 (type: int)
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
Execution mode: vectorized, llap
LLAP IO: may be used (ACID table)
Map 8
@@ -3424,15 +3428,15 @@ STAGE PLANS:
condition map:
Right Outer Join 0 to 1
keys:
- 0 a (type: int)
+ 0 _col1 (type: int)
1 a (type: int)
- outputColumnNames: _col0, _col4, _col6, _col7
+ outputColumnNames: _col0, _col1, _col3, _col4
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Filter Operator
- predicate: ((_col6 > 8) and (_col0 = _col6)) (type: boolean)
+ predicate: ((_col3 > 8) and (_col1 = _col3)) (type: boolean)
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: _col4 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
outputColumnNames: _col0
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Reduce Output Operator
@@ -3442,10 +3446,10 @@ STAGE PLANS:
Map-reduce partition columns: UDFToInteger(_col0) (type: int)
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Filter Operator
- predicate: ((_col6 <= 8) and (_col0 = _col6)) (type: boolean)
+ predicate: ((_col3 <= 8) and (_col1 = _col3)) (type: boolean)
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: _col4 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
outputColumnNames: _col0
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Reduce Output Operator
@@ -3455,10 +3459,10 @@ STAGE PLANS:
Map-reduce partition columns: UDFToInteger(_col0) (type: int)
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Filter Operator
- predicate: ((_col6 <= 8) and (_col0 = _col6)) (type: boolean)
+ predicate: ((_col3 <= 8) and (_col1 = _col3)) (type: boolean)
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: _col0 (type: int)
+ expressions: _col1 (type: int)
outputColumnNames: _col0
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Reduce Output Operator
@@ -3468,15 +3472,15 @@ STAGE PLANS:
Map-reduce partition columns: _col0 (type: int)
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Filter Operator
- predicate: (_col0 = _col6) (type: boolean)
+ predicate: (_col1 = _col3) (type: boolean)
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: _col4 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
- outputColumnNames: _col4
+ expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ outputColumnNames: _col0
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Group By Operator
aggregations: count()
- keys: _col4 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ keys: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
minReductionHashAggr: 0.99
mode: hash
outputColumnNames: _col0, _col1
@@ -3489,10 +3493,10 @@ STAGE PLANS:
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
value expressions: _col1 (type: bigint)
Filter Operator
- predicate: _col0 is null (type: boolean)
+ predicate: _col1 is null (type: boolean)
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: _col6 (type: int), _col7 (type: int)
+ expressions: _col3 (type: int), _col4 (type: int)
outputColumnNames: _col0, _col1
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Reduce Output Operator
@@ -3720,32 +3724,30 @@ STAGE PLANS:
Tez
#### A masked pattern was here ####
Edges:
- Map 1 <- Reducer 9 (BROADCAST_EDGE)
Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 8 (SIMPLE_EDGE)
Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
Reducer 4 <- Reducer 2 (SIMPLE_EDGE)
Reducer 5 <- Reducer 2 (SIMPLE_EDGE)
Reducer 6 <- Reducer 2 (SIMPLE_EDGE)
Reducer 7 <- Reducer 2 (SIMPLE_EDGE)
- Reducer 9 <- Map 8 (CUSTOM_SIMPLE_EDGE)
#### A masked pattern was here ####
Vertices:
Map 1
Map Operator Tree:
TableScan
- alias: t
- filterExpr: (a BETWEEN DynamicValue(RS_3_s_a_min) AND DynamicValue(RS_3_s_a_max) and in_bloom_filter(a, DynamicValue(RS_3_s_a_bloom_filter))) (type: boolean)
+ alias: acidtbl
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
- Filter Operator
- predicate: (a BETWEEN DynamicValue(RS_3_s_a_min) AND DynamicValue(RS_3_s_a_max) and in_bloom_filter(a, DynamicValue(RS_3_s_a_bloom_filter))) (type: boolean)
+ Select Operator
+ expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), a (type: int)
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Reduce Output Operator
- key expressions: a (type: int)
+ key expressions: _col1 (type: int)
null sort order: z
sort order: +
- Map-reduce partition columns: a (type: int)
+ Map-reduce partition columns: _col1 (type: int)
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
- value expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ value expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
Execution mode: vectorized, llap
LLAP IO: may be used (ACID table)
Map 8
@@ -3760,21 +3762,6 @@ STAGE PLANS:
Map-reduce partition columns: a (type: int)
Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
value expressions: b (type: int)
- Select Operator
- expressions: a (type: int)
- outputColumnNames: a
- Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
- Group By Operator
- aggregations: min(a), max(a), bloom_filter(a, expectedEntries=1000)
- minReductionHashAggr: 0.99
- mode: hash
- outputColumnNames: _col0, _col1, _col2
- Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: NONE
- Reduce Output Operator
- null sort order:
- sort order:
- Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary)
Execution mode: vectorized, llap
LLAP IO: all inputs
Reducer 2
@@ -3784,15 +3771,15 @@ STAGE PLANS:
condition map:
Right Outer Join 0 to 1
keys:
- 0 a (type: int)
+ 0 _col1 (type: int)
1 a (type: int)
- outputColumnNames: _col0, _col4, _col6, _col7
+ outputColumnNames: _col0, _col1, _col3, _col4
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Filter Operator
- predicate: ((_col6 > 8) and (_col0 = _col6)) (type: boolean)
+ predicate: ((_col3 > 8) and (_col1 = _col3)) (type: boolean)
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: _col4 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
outputColumnNames: _col0
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Reduce Output Operator
@@ -3802,10 +3789,10 @@ STAGE PLANS:
Map-reduce partition columns: UDFToInteger(_col0) (type: int)
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Filter Operator
- predicate: ((_col6 <= 8) and (_col0 = _col6)) (type: boolean)
+ predicate: ((_col3 <= 8) and (_col1 = _col3)) (type: boolean)
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: _col4 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
outputColumnNames: _col0
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Reduce Output Operator
@@ -3815,10 +3802,10 @@ STAGE PLANS:
Map-reduce partition columns: UDFToInteger(_col0) (type: int)
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Filter Operator
- predicate: ((_col6 <= 8) and (_col0 = _col6)) (type: boolean)
+ predicate: ((_col3 <= 8) and (_col1 = _col3)) (type: boolean)
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: _col0 (type: int)
+ expressions: _col1 (type: int)
outputColumnNames: _col0
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Reduce Output Operator
@@ -3828,15 +3815,15 @@ STAGE PLANS:
Map-reduce partition columns: _col0 (type: int)
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Filter Operator
- predicate: (_col0 = _col6) (type: boolean)
+ predicate: (_col1 = _col3) (type: boolean)
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: _col4 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
- outputColumnNames: _col4
+ expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ outputColumnNames: _col0
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Group By Operator
aggregations: count()
- keys: _col4 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ keys: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
minReductionHashAggr: 0.99
mode: hash
outputColumnNames: _col0, _col1
@@ -3849,10 +3836,10 @@ STAGE PLANS:
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
value expressions: _col1 (type: bigint)
Filter Operator
- predicate: _col0 is null (type: boolean)
+ predicate: _col1 is null (type: boolean)
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: _col6 (type: int), _col7 (type: int)
+ expressions: _col3 (type: int), _col4 (type: int)
outputColumnNames: _col0, _col1
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Reduce Output Operator
@@ -3950,19 +3937,6 @@ STAGE PLANS:
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.acidtbl
Write Type: INSERT
- Reducer 9
- Execution mode: vectorized, llap
- Reduce Operator Tree:
- Group By Operator
- aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=1000)
- mode: final
- outputColumnNames: _col0, _col1, _col2
- Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: NONE
- Reduce Output Operator
- null sort order:
- sort order:
- Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary)
Stage: Stage-6
Dependency Collection
diff --git a/ql/src/test/results/clientpositive/llap/sort_acid.q.out b/ql/src/test/results/clientpositive/llap/sort_acid.q.out
index ae7b2ce2741..285bb249e69 100644
--- a/ql/src/test/results/clientpositive/llap/sort_acid.q.out
+++ b/ql/src/test/results/clientpositive/llap/sort_acid.q.out
@@ -287,13 +287,13 @@ POSTHOOK: Output: default@acidtlb
POSTHOOK: Output: default@acidtlb
POSTHOOK: Output: default@merge_tmp_table
CBO PLAN:
-HiveProject(col0=[$3], col1=[$0], col2=[$1], col3=[$2])
- HiveJoin(condition=[=($2, $0)], joinType=[left], algorithm=[none], cost=[not available])
+HiveProject(col0=[$2], col1=[$0], col2=[$1], col3=[$3])
+ HiveJoin(condition=[=($3, $0)], joinType=[left], algorithm=[none], cost=[not available])
HiveProject(c=[$0], d=[$1])
HiveTableScan(table=[[default, othertlb]], table:alias=[s])
- HiveProject(a=[$0], ROW__ID=[$4])
+ HiveProject(row__id=[$4], a=[$0])
HiveFilter(condition=[IS NOT NULL($0)])
- HiveTableScan(table=[[default, acidtlb]], table:alias=[t])
+ HiveTableScan(table=[[default, acidtlb]], table:alias=[acidtlb])
PREHOOK: query: merge into acidtlb as t using othertlb as s on t.a = s.c
when matched and s.c < 30 then delete
@@ -319,7 +319,7 @@ POSTHOOK: Lineage: acidtlb.a SIMPLE [(othertlb)s.FieldSchema(name:c, type:int, c
POSTHOOK: Lineage: acidtlb.a SIMPLE [(othertlb)s.FieldSchema(name:c, type:int, comment:null), ]
POSTHOOK: Lineage: acidtlb.b EXPRESSION [(othertlb)s.FieldSchema(name:d, type:int, comment:null), ]
POSTHOOK: Lineage: acidtlb.b EXPRESSION [(othertlb)s.FieldSchema(name:d, type:int, comment:null), ]
-POSTHOOK: Lineage: merge_tmp_table.val EXPRESSION [(acidtlb)t.FieldSchema(name:ROW__ID, type:struct<writeId:bigint,bucketId:int,rowId:bigint>, comment:), ]
+POSTHOOK: Lineage: merge_tmp_table.val EXPRESSION [(acidtlb)acidtlb.FieldSchema(name:ROW__ID, type:struct<writeId:bigint,bucketId:int,rowId:bigint>, comment:), ]
PREHOOK: query: select * from acidtlb
PREHOOK: type: QUERY
PREHOOK: Input: default@acidtlb
diff --git a/ql/src/test/results/clientpositive/llap/sqlmerge.q.out b/ql/src/test/results/clientpositive/llap/sqlmerge.q.out
index 9fd6a978c24..ff8c773ccf2 100644
--- a/ql/src/test/results/clientpositive/llap/sqlmerge.q.out
+++ b/ql/src/test/results/clientpositive/llap/sqlmerge.q.out
@@ -82,23 +82,23 @@ STAGE PLANS:
Map 8
Map Operator Tree:
TableScan
- alias: t
+ alias: acidtbl_n0
filterExpr: a is not null (type: boolean)
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Filter Operator
predicate: a is not null (type: boolean)
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: a (type: int), ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), a (type: int)
outputColumnNames: _col0, _col1
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Reduce Output Operator
- key expressions: _col0 (type: int)
+ key expressions: _col1 (type: int)
null sort order: z
sort order: +
- Map-reduce partition columns: _col0 (type: int)
+ Map-reduce partition columns: _col1 (type: int)
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col1 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ value expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
Execution mode: vectorized, llap
LLAP IO: may be used (ACID table)
Reducer 2
@@ -109,11 +109,11 @@ STAGE PLANS:
Left Outer Join 0 to 1
keys:
0 _col0 (type: int)
- 1 _col0 (type: int)
+ 1 _col1 (type: int)
outputColumnNames: _col0, _col1, _col2, _col3
Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: _col3 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col1 (type: int), _col2 (type: int), _col0 (type: int)
+ expressions: _col2 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col1 (type: int), _col3 (type: int), _col0 (type: int)
outputColumnNames: _col0, _col1, _col2, _col3
Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
Filter Operator
@@ -387,7 +387,7 @@ STAGE PLANS:
Map 1
Map Operator Tree:
TableScan
- alias: t
+ alias: acidtbl_n0
filterExpr: a is not null (type: boolean)
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Filter Operator
@@ -588,23 +588,23 @@ STAGE PLANS:
Map 7
Map Operator Tree:
TableScan
- alias: t
+ alias: acidtbl_n0
filterExpr: a is not null (type: boolean)
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Filter Operator
predicate: a is not null (type: boolean)
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: a (type: int), ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), a (type: int)
outputColumnNames: _col0, _col1
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Reduce Output Operator
- key expressions: _col0 (type: int)
+ key expressions: _col1 (type: int)
null sort order: z
sort order: +
- Map-reduce partition columns: _col0 (type: int)
+ Map-reduce partition columns: _col1 (type: int)
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col1 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ value expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
Execution mode: vectorized, llap
LLAP IO: may be used (ACID table)
Reducer 2
@@ -615,11 +615,11 @@ STAGE PLANS:
Left Outer Join 0 to 1
keys:
0 _col0 (type: int)
- 1 _col0 (type: int)
+ 1 _col1 (type: int)
outputColumnNames: _col0, _col1, _col2, _col3
Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: _col3 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col1 (type: int), _col2 (type: int), _col0 (type: int)
+ expressions: _col2 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col1 (type: int), _col3 (type: int), _col0 (type: int)
outputColumnNames: _col0, _col1, _col2, _col3
Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
Filter Operator
diff --git a/ql/src/test/results/clientpositive/llap/sqlmerge_stats.q.out b/ql/src/test/results/clientpositive/llap/sqlmerge_stats.q.out
index f6b36362ed9..b757bbc9d77 100644
--- a/ql/src/test/results/clientpositive/llap/sqlmerge_stats.q.out
+++ b/ql/src/test/results/clientpositive/llap/sqlmerge_stats.q.out
@@ -174,16 +174,16 @@ STAGE PLANS:
predicate: a is not null (type: boolean)
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
Select Operator
- expressions: a (type: int), ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), a (type: int)
outputColumnNames: _col0, _col1
Statistics: Num rows: 1 Data size: 80 Basic stats: COMPLETE Column stats: COMPLETE
Reduce Output Operator
- key expressions: _col0 (type: int)
+ key expressions: _col1 (type: int)
null sort order: z
sort order: +
- Map-reduce partition columns: _col0 (type: int)
+ Map-reduce partition columns: _col1 (type: int)
Statistics: Num rows: 1 Data size: 80 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: _col1 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ value expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
Execution mode: vectorized, llap
LLAP IO: may be used (ACID table)
Reducer 2
@@ -194,74 +194,70 @@ STAGE PLANS:
Left Outer Join 0 to 1
keys:
0 _col0 (type: int)
- 1 _col0 (type: int)
+ 1 _col1 (type: int)
outputColumnNames: _col0, _col1, _col2, _col3
Statistics: Num rows: 2 Data size: 176 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: _col0 (type: int), _col1 (type: int), _col3 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col2 (type: int)
- outputColumnNames: _col0, _col1, _col2, _col3
- Statistics: Num rows: 2 Data size: 176 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator
- predicate: (_col3 = _col0) (type: boolean)
- Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: _col2 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
- outputColumnNames: _col0
+ Filter Operator
+ predicate: (_col3 = _col0) (type: boolean)
+ Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: _col2 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: UDFToInteger(_col0) (type: int)
Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- key expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
- null sort order: z
- sort order: +
- Map-reduce partition columns: UDFToInteger(_col0) (type: int)
- Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator
- predicate: (_col3 = _col0) (type: boolean)
- Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: _col3 (type: int)
- outputColumnNames: _col0
+ Filter Operator
+ predicate: (_col3 = _col0) (type: boolean)
+ Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: _col3 (type: int)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: int)
+ null sort order: a
+ sort order: +
+ Map-reduce partition columns: _col0 (type: int)
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- key expressions: _col0 (type: int)
- null sort order: a
- sort order: +
- Map-reduce partition columns: _col0 (type: int)
- Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator
- predicate: _col3 is null (type: boolean)
+ Filter Operator
+ predicate: _col3 is null (type: boolean)
+ Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: _col0 (type: int), _col1 (type: int)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: int)
+ null sort order: a
+ sort order: +
+ Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col1 (type: int)
+ Filter Operator
+ predicate: (_col3 = _col0) (type: boolean)
+ Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: _col2 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ outputColumnNames: _col2
Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: _col0 (type: int), _col1 (type: int)
+ Group By Operator
+ aggregations: count()
+ keys: _col2 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ minReductionHashAggr: 0.4
+ mode: hash
outputColumnNames: _col0, _col1
- Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE
Reduce Output Operator
- key expressions: _col0 (type: int)
- null sort order: a
+ key expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ null sort order: z
sort order: +
- Map-reduce partition columns: _col0 (type: int)
- Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: _col1 (type: int)
- Filter Operator
- predicate: (_col3 = _col0) (type: boolean)
- Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: _col2 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
- outputColumnNames: _col2
- Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
- Group By Operator
- aggregations: count()
- keys: _col2 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
- minReductionHashAggr: 0.4
- mode: hash
- outputColumnNames: _col0, _col1
+ Map-reduce partition columns: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- key expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
- null sort order: z
- sort order: +
- Map-reduce partition columns: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
- Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: _col1 (type: bigint)
+ value expressions: _col1 (type: bigint)
Reducer 3
Execution mode: vectorized, llap
Reduce Operator Tree:
@@ -708,23 +704,23 @@ STAGE PLANS:
Map 1
Map Operator Tree:
TableScan
- alias: t
+ alias: t2
filterExpr: a is not null (type: boolean)
Statistics: Num rows: 4 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
Filter Operator
predicate: a is not null (type: boolean)
Statistics: Num rows: 4 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
Select Operator
- expressions: a (type: int), c (type: int), ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), a (type: int), c (type: int)
outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 4 Data size: 336 Basic stats: COMPLETE Column stats: COMPLETE
Reduce Output Operator
- key expressions: _col0 (type: int)
+ key expressions: _col1 (type: int)
null sort order: z
sort order: +
- Map-reduce partition columns: _col0 (type: int)
+ Map-reduce partition columns: _col1 (type: int)
Statistics: Num rows: 4 Data size: 336 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: _col1 (type: int), _col2 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ value expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col2 (type: int)
Execution mode: vectorized, llap
LLAP IO: may be used (ACID table)
Map 7
@@ -752,12 +748,12 @@ STAGE PLANS:
condition map:
Right Outer Join 0 to 1
keys:
- 0 _col0 (type: int)
+ 0 _col1 (type: int)
1 _col0 (type: int)
outputColumnNames: _col0, _col1, _col2, _col3, _col4
Statistics: Num rows: 4 Data size: 284 Basic stats: COMPLETE Column stats: COMPLETE
Select Operator
- expressions: _col3 (type: int), _col4 (type: int), _col2 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col1 (type: int), _col0 (type: int)
+ expressions: _col3 (type: int), _col4 (type: int), _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col2 (type: int), _col1 (type: int)
outputColumnNames: _col0, _col1, _col2, _col3, _col4
Statistics: Num rows: 4 Data size: 284 Basic stats: COMPLETE Column stats: COMPLETE
Filter Operator
@@ -975,7 +971,7 @@ POSTHOOK: Input: default@upd_t2_1
POSTHOOK: Output: default@merge_tmp_table
POSTHOOK: Output: default@t2
POSTHOOK: Output: default@t2
-POSTHOOK: Lineage: merge_tmp_table.val EXPRESSION [(t2)t.FieldSchema(name:ROW__ID, type:struct<writeId:bigint,bucketId:int,rowId:bigint>, comment:), ]
+POSTHOOK: Lineage: merge_tmp_table.val EXPRESSION [(t2)t2.FieldSchema(name:ROW__ID, type:struct<writeId:bigint,bucketId:int,rowId:bigint>, comment:), ]
POSTHOOK: Lineage: t2.a SIMPLE [(upd_t2_1)u.FieldSchema(name:a, type:int, comment:null), ]
POSTHOOK: Lineage: t2.a SIMPLE [(upd_t2_1)u.FieldSchema(name:a, type:int, comment:null), ]
POSTHOOK: Lineage: t2.b SIMPLE [(upd_t2_1)u.FieldSchema(name:b, type:int, comment:null), ]
@@ -1027,23 +1023,23 @@ STAGE PLANS:
Map 1
Map Operator Tree:
TableScan
- alias: t
+ alias: t2
filterExpr: a is not null (type: boolean)
Statistics: Num rows: 5 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
Filter Operator
predicate: a is not null (type: boolean)
Statistics: Num rows: 5 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
Select Operator
- expressions: a (type: int), c (type: int), ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), a (type: int), c (type: int)
outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 5 Data size: 420 Basic stats: COMPLETE Column stats: COMPLETE
Reduce Output Operator
- key expressions: _col0 (type: int)
+ key expressions: _col1 (type: int)
null sort order: z
sort order: +
- Map-reduce partition columns: _col0 (type: int)
+ Map-reduce partition columns: _col1 (type: int)
Statistics: Num rows: 5 Data size: 420 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: _col1 (type: int), _col2 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ value expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col2 (type: int)
Execution mode: vectorized, llap
LLAP IO: may be used (ACID table)
Map 7
@@ -1071,12 +1067,12 @@ STAGE PLANS:
condition map:
Right Outer Join 0 to 1
keys:
- 0 _col0 (type: int)
+ 0 _col1 (type: int)
1 _col0 (type: int)
outputColumnNames: _col0, _col1, _col2, _col3, _col4
Statistics: Num rows: 4 Data size: 284 Basic stats: COMPLETE Column stats: COMPLETE
Select Operator
- expressions: _col3 (type: int), _col4 (type: int), _col2 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col1 (type: int), _col0 (type: int)
+ expressions: _col3 (type: int), _col4 (type: int), _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col2 (type: int), _col1 (type: int)
outputColumnNames: _col0, _col1, _col2, _col3, _col4
Statistics: Num rows: 4 Data size: 284 Basic stats: COMPLETE Column stats: COMPLETE
Filter Operator
@@ -1294,7 +1290,7 @@ POSTHOOK: Input: default@upd_t2_2
POSTHOOK: Output: default@merge_tmp_table
POSTHOOK: Output: default@t2
POSTHOOK: Output: default@t2
-POSTHOOK: Lineage: merge_tmp_table.val EXPRESSION [(t2)t.FieldSchema(name:ROW__ID, type:struct<writeId:bigint,bucketId:int,rowId:bigint>, comment:), ]
+POSTHOOK: Lineage: merge_tmp_table.val EXPRESSION [(t2)t2.FieldSchema(name:ROW__ID, type:struct<writeId:bigint,bucketId:int,rowId:bigint>, comment:), ]
POSTHOOK: Lineage: t2.a SIMPLE [(upd_t2_2)u.FieldSchema(name:a, type:int, comment:null), ]
POSTHOOK: Lineage: t2.a SIMPLE [(upd_t2_2)u.FieldSchema(name:a, type:int, comment:null), ]
POSTHOOK: Lineage: t2.b SIMPLE [(upd_t2_2)u.FieldSchema(name:b, type:int, comment:null), ]
@@ -1346,23 +1342,23 @@ STAGE PLANS:
Map 1
Map Operator Tree:
TableScan
- alias: t
+ alias: t2
filterExpr: a is not null (type: boolean)
Statistics: Num rows: 6 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
Filter Operator
predicate: a is not null (type: boolean)
Statistics: Num rows: 6 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
Select Operator
- expressions: a (type: int), c (type: int), ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), a (type: int), c (type: int)
outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 6 Data size: 504 Basic stats: COMPLETE Column stats: COMPLETE
Reduce Output Operator
- key expressions: _col0 (type: int)
+ key expressions: _col1 (type: int)
null sort order: z
sort order: +
- Map-reduce partition columns: _col0 (type: int)
+ Map-reduce partition columns: _col1 (type: int)
Statistics: Num rows: 6 Data size: 504 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: _col1 (type: int), _col2 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ value expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col2 (type: int)
Execution mode: vectorized, llap
LLAP IO: may be used (ACID table)
Map 7
@@ -1390,12 +1386,12 @@ STAGE PLANS:
condition map:
Right Outer Join 0 to 1
keys:
- 0 _col0 (type: int)
+ 0 _col1 (type: int)
1 _col0 (type: int)
outputColumnNames: _col0, _col1, _col2, _col3, _col4
Statistics: Num rows: 5 Data size: 376 Basic stats: COMPLETE Column stats: COMPLETE
Select Operator
- expressions: _col3 (type: int), _col4 (type: int), _col2 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col1 (type: int), _col0 (type: int)
+ expressions: _col3 (type: int), _col4 (type: int), _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col2 (type: int), _col1 (type: int)
outputColumnNames: _col0, _col1, _col2, _col3, _col4
Statistics: Num rows: 5 Data size: 376 Basic stats: COMPLETE Column stats: COMPLETE
Filter Operator
@@ -1613,7 +1609,7 @@ POSTHOOK: Input: default@upd_t2_3
POSTHOOK: Output: default@merge_tmp_table
POSTHOOK: Output: default@t2
POSTHOOK: Output: default@t2
-POSTHOOK: Lineage: merge_tmp_table.val EXPRESSION [(t2)t.FieldSchema(name:ROW__ID, type:struct<writeId:bigint,bucketId:int,rowId:bigint>, comment:), ]
+POSTHOOK: Lineage: merge_tmp_table.val EXPRESSION [(t2)t2.FieldSchema(name:ROW__ID, type:struct<writeId:bigint,bucketId:int,rowId:bigint>, comment:), ]
POSTHOOK: Lineage: t2.a SIMPLE [(upd_t2_3)u.FieldSchema(name:a, type:int, comment:null), ]
POSTHOOK: Lineage: t2.a SIMPLE [(upd_t2_3)u.FieldSchema(name:a, type:int, comment:null), ]
POSTHOOK: Lineage: t2.b SIMPLE [(upd_t2_3)u.FieldSchema(name:b, type:int, comment:null), ]
@@ -1665,23 +1661,23 @@ STAGE PLANS:
Map 1
Map Operator Tree:
TableScan
- alias: t
+ alias: t2
filterExpr: a is not null (type: boolean)
Statistics: Num rows: 7 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE
Filter Operator
predicate: a is not null (type: boolean)
Statistics: Num rows: 7 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE
Select Operator
- expressions: a (type: int), c (type: int), ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), a (type: int), c (type: int)
outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 7 Data size: 588 Basic stats: COMPLETE Column stats: COMPLETE
Reduce Output Operator
- key expressions: _col0 (type: int)
+ key expressions: _col1 (type: int)
null sort order: z
sort order: +
- Map-reduce partition columns: _col0 (type: int)
+ Map-reduce partition columns: _col1 (type: int)
Statistics: Num rows: 7 Data size: 588 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: _col1 (type: int), _col2 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ value expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col2 (type: int)
Execution mode: vectorized, llap
LLAP IO: may be used (ACID table)
Map 7
@@ -1709,12 +1705,12 @@ STAGE PLANS:
condition map:
Right Outer Join 0 to 1
keys:
- 0 _col0 (type: int)
+ 0 _col1 (type: int)
1 _col0 (type: int)
outputColumnNames: _col0, _col1, _col2, _col3, _col4
Statistics: Num rows: 5 Data size: 376 Basic stats: COMPLETE Column stats: COMPLETE
Select Operator
- expressions: _col3 (type: int), _col4 (type: int), _col2 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col1 (type: int), _col0 (type: int)
+ expressions: _col3 (type: int), _col4 (type: int), _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col2 (type: int), _col1 (type: int)
outputColumnNames: _col0, _col1, _col2, _col3, _col4
Statistics: Num rows: 5 Data size: 376 Basic stats: COMPLETE Column stats: COMPLETE
Filter Operator
@@ -1932,7 +1928,7 @@ POSTHOOK: Input: default@upd_t2_4
POSTHOOK: Output: default@merge_tmp_table
POSTHOOK: Output: default@t2
POSTHOOK: Output: default@t2
-POSTHOOK: Lineage: merge_tmp_table.val EXPRESSION [(t2)t.FieldSchema(name:ROW__ID, type:struct<writeId:bigint,bucketId:int,rowId:bigint>, comment:), ]
+POSTHOOK: Lineage: merge_tmp_table.val EXPRESSION [(t2)t2.FieldSchema(name:ROW__ID, type:struct<writeId:bigint,bucketId:int,rowId:bigint>, comment:), ]
POSTHOOK: Lineage: t2.a SIMPLE [(upd_t2_4)u.FieldSchema(name:a, type:int, comment:null), ]
POSTHOOK: Lineage: t2.a SIMPLE [(upd_t2_4)u.FieldSchema(name:a, type:int, comment:null), ]
POSTHOOK: Lineage: t2.b SIMPLE [(upd_t2_4)u.FieldSchema(name:b, type:int, comment:null), ]
@@ -2051,23 +2047,23 @@ STAGE PLANS:
Map 6
Map Operator Tree:
TableScan
- alias: t
+ alias: t3
filterExpr: a is not null (type: boolean)
Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
Filter Operator
predicate: a is not null (type: boolean)
Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
Select Operator
- expressions: a (type: int), ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), a (type: int)
outputColumnNames: _col0, _col1
Statistics: Num rows: 2 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE
Reduce Output Operator
- key expressions: _col0 (type: int)
+ key expressions: _col1 (type: int)
null sort order: z
sort order: +
- Map-reduce partition columns: _col0 (type: int)
+ Map-reduce partition columns: _col1 (type: int)
Statistics: Num rows: 2 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: _col1 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ value expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
Execution mode: vectorized, llap
LLAP IO: may be used (ACID table)
Reducer 2
@@ -2078,60 +2074,56 @@ STAGE PLANS:
Left Outer Join 0 to 1
keys:
0 _col0 (type: int)
- 1 _col0 (type: int)
+ 1 _col1 (type: int)
outputColumnNames: _col0, _col1, _col2, _col3
Statistics: Num rows: 2 Data size: 176 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: _col0 (type: int), _col1 (type: int), _col3 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col2 (type: int)
- outputColumnNames: _col0, _col1, _col2, _col3
- Statistics: Num rows: 2 Data size: 176 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator
- predicate: (_col3 = _col0) (type: boolean)
- Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: _col2 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
- outputColumnNames: _col0
+ Filter Operator
+ predicate: (_col3 = _col0) (type: boolean)
+ Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: _col2 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: UDFToInteger(_col0) (type: int)
Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: _col3 is null (type: boolean)
+ Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: _col1 (type: int)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: int)
+ null sort order: a
+ sort order: +
+ Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: (_col3 = _col0) (type: boolean)
+ Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: _col2 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ outputColumnNames: _col2
+ Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
+ Group By Operator
+ aggregations: count()
+ keys: _col2 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ minReductionHashAggr: 0.4
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE
Reduce Output Operator
key expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
null sort order: z
sort order: +
- Map-reduce partition columns: UDFToInteger(_col0) (type: int)
- Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator
- predicate: _col3 is null (type: boolean)
- Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: _col1 (type: int)
- outputColumnNames: _col0
- Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- key expressions: _col0 (type: int)
- null sort order: a
- sort order: +
- Map-reduce partition columns: _col0 (type: int)
- Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator
- predicate: (_col3 = _col0) (type: boolean)
- Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: _col2 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
- outputColumnNames: _col2
- Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
- Group By Operator
- aggregations: count()
- keys: _col2 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
- minReductionHashAggr: 0.4
- mode: hash
- outputColumnNames: _col0, _col1
+ Map-reduce partition columns: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- key expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
- null sort order: z
- sort order: +
- Map-reduce partition columns: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
- Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: _col1 (type: bigint)
+ value expressions: _col1 (type: bigint)
Reducer 3
Execution mode: vectorized, llap
Reduce Operator Tree:
@@ -2254,7 +2246,7 @@ POSTHOOK: Input: default@upd_t3
POSTHOOK: Output: default@merge_tmp_table
POSTHOOK: Output: default@t3
POSTHOOK: Output: default@t3
-POSTHOOK: Lineage: merge_tmp_table.val EXPRESSION [(t3)t.FieldSchema(name:ROW__ID, type:struct<writeId:bigint,bucketId:int,rowId:bigint>, comment:), ]
+POSTHOOK: Lineage: merge_tmp_table.val EXPRESSION [(t3)t3.FieldSchema(name:ROW__ID, type:struct<writeId:bigint,bucketId:int,rowId:bigint>, comment:), ]
POSTHOOK: Lineage: t3.a SIMPLE [(upd_t3)u.FieldSchema(name:b, type:int, comment:null), ]
POSTHOOK: Lineage: t3.b SIMPLE []
PREHOOK: query: select * from t3
@@ -2366,23 +2358,23 @@ STAGE PLANS:
Map 7
Map Operator Tree:
TableScan
- alias: t
+ alias: t4
filterExpr: a is not null (type: boolean)
Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
Filter Operator
predicate: a is not null (type: boolean)
Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
Select Operator
- expressions: a (type: int), ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), a (type: int)
outputColumnNames: _col0, _col1
Statistics: Num rows: 2 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE
Reduce Output Operator
- key expressions: _col0 (type: int)
+ key expressions: _col1 (type: int)
null sort order: z
sort order: +
- Map-reduce partition columns: _col0 (type: int)
+ Map-reduce partition columns: _col1 (type: int)
Statistics: Num rows: 2 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: _col1 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ value expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
Execution mode: vectorized, llap
LLAP IO: may be used (ACID table)
Reducer 2
@@ -2393,73 +2385,69 @@ STAGE PLANS:
Left Outer Join 0 to 1
keys:
0 _col0 (type: int)
- 1 _col0 (type: int)
+ 1 _col1 (type: int)
outputColumnNames: _col0, _col1, _col2, _col3
Statistics: Num rows: 2 Data size: 176 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: _col0 (type: int), _col1 (type: int), _col3 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col2 (type: int)
- outputColumnNames: _col0, _col1, _col2, _col3
- Statistics: Num rows: 2 Data size: 176 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator
- predicate: (_col3 = _col0) (type: boolean)
- Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: _col2 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
- outputColumnNames: _col0
+ Filter Operator
+ predicate: (_col3 = _col0) (type: boolean)
+ Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: _col2 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: UDFToInteger(_col0) (type: int)
Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- key expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
- null sort order: z
- sort order: +
- Map-reduce partition columns: UDFToInteger(_col0) (type: int)
- Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator
- predicate: (_col3 = _col0) (type: boolean)
- Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: _col3 (type: int)
- outputColumnNames: _col0
+ Filter Operator
+ predicate: (_col3 = _col0) (type: boolean)
+ Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: _col3 (type: int)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: int)
+ null sort order: a
+ sort order: +
+ Map-reduce partition columns: _col0 (type: int)
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- key expressions: _col0 (type: int)
- null sort order: a
- sort order: +
- Map-reduce partition columns: _col0 (type: int)
- Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator
- predicate: _col3 is null (type: boolean)
- Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: _col1 (type: int)
- outputColumnNames: _col0
+ Filter Operator
+ predicate: _col3 is null (type: boolean)
+ Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: _col1 (type: int)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: int)
+ null sort order: a
+ sort order: +
+ Map-reduce partition columns: _col0 (type: int)
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: (_col3 = _col0) (type: boolean)
+ Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: _col2 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ outputColumnNames: _col2
+ Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
+ Group By Operator
+ aggregations: count()
+ keys: _col2 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ minReductionHashAggr: 0.4
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE
Reduce Output Operator
- key expressions: _col0 (type: int)
- null sort order: a
+ key expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ null sort order: z
sort order: +
- Map-reduce partition columns: _col0 (type: int)
- Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator
- predicate: (_col3 = _col0) (type: boolean)
- Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: _col2 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
- outputColumnNames: _col2
- Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
- Group By Operator
- aggregations: count()
- keys: _col2 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
- minReductionHashAggr: 0.4
- mode: hash
- outputColumnNames: _col0, _col1
+ Map-reduce partition columns: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- key expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
- null sort order: z
- sort order: +
- Map-reduce partition columns: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
- Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: _col1 (type: bigint)
+ value expressions: _col1 (type: bigint)
Reducer 3
Execution mode: vectorized, llap
Reduce Operator Tree:
@@ -2613,7 +2601,7 @@ POSTHOOK: Input: default@upd_t4
POSTHOOK: Output: default@merge_tmp_table
POSTHOOK: Output: default@t4
POSTHOOK: Output: default@t4
-POSTHOOK: Lineage: merge_tmp_table.val EXPRESSION [(t4)t.FieldSchema(name:ROW__ID, type:struct<writeId:bigint,bucketId:int,rowId:bigint>, comment:), ]
+POSTHOOK: Lineage: merge_tmp_table.val EXPRESSION [(t4)t4.FieldSchema(name:ROW__ID, type:struct<writeId:bigint,bucketId:int,rowId:bigint>, comment:), ]
POSTHOOK: Lineage: t4.a SIMPLE [(upd_t4)u.FieldSchema(name:b, type:int, comment:null), ]
POSTHOOK: Lineage: t4.a SIMPLE [(upd_t4)u.FieldSchema(name:b, type:int, comment:null), ]
POSTHOOK: Lineage: t4.b SIMPLE []