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/06/08 14:19:59 UTC
[hive] branch master updated: HIVE-21160: Rewrite Update statement as Multi-insert and do Update split early (Krisztian Kasa, reviewed by Peter Vary, Zoltan Haindrich)
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 3048d409261 HIVE-21160: Rewrite Update statement as Multi-insert and do Update split early (Krisztian Kasa, reviewed by Peter Vary, Zoltan Haindrich)
3048d409261 is described below
commit 3048d4092611591250d68093035ec96c0af406da
Author: Krisztian Kasa <kk...@cloudera.com>
AuthorDate: Wed Jun 8 16:19:47 2022 +0200
HIVE-21160: Rewrite Update statement as Multi-insert and do Update split early (Krisztian Kasa, reviewed by Peter Vary, Zoltan Haindrich)
---
.../java/org/apache/hadoop/hive/conf/HiveConf.java | 13 +-
.../org/apache/hadoop/hive/ql/TestAcidOnTez.java | 2 +-
.../java/org/apache/hive/jdbc/TestJdbcDriver2.java | 40 +
ql/src/java/org/apache/hadoop/hive/ql/Context.java | 85 +-
.../hadoop/hive/ql/exec/FileSinkOperator.java | 9 +-
.../hive/ql/parse/AcidExportSemanticAnalyzer.java | 34 +-
.../hive/ql/parse/MergeSemanticAnalyzer.java | 193 ++---
.../hive/ql/parse/RewriteSemanticAnalyzer.java | 215 ++++-
.../hadoop/hive/ql/parse/SemanticAnalyzer.java | 13 +-
.../ql/parse/UpdateDeleteSemanticAnalyzer.java | 185 +++--
.../apache/hadoop/hive/ql/plan/FileSinkDesc.java | 18 +-
.../apache/hadoop/hive/ql/TestTxnCommands2.java | 8 +-
.../apache/hadoop/hive/ql/TestTxnCommands3.java | 24 +-
.../apache/hadoop/hive/ql/TestTxnConcatenate.java | 18 +-
.../org/apache/hadoop/hive/ql/TestTxnExIm.java | 6 +-
.../org/apache/hadoop/hive/ql/TestTxnLoadData.java | 16 +-
.../apache/hadoop/hive/ql/TestTxnNoBuckets.java | 34 +-
.../hadoop/hive/ql/TxnCommandsBaseForTests.java | 2 +-
.../hadoop/hive/ql/exec/TestFileSinkOperator.java | 2 +-
.../hadoop/hive/ql/lockmgr/TestDbTxnManager2.java | 44 +-
.../clientpositive/bucket_num_reducers_acid.q | 9 +-
ql/src/test/queries/clientpositive/split_update.q | 14 +
.../authorization_update_noupdatepriv.q.out | 2 +-
.../clientnegative/update_notnull_constraint.q.out | 28 +-
.../llap/acid_direct_update_delete.q.out | 8 +
.../acid_direct_update_delete_partitions.q.out | 36 +
.../llap/acid_insert_overwrite_update.q.out | 34 +-
.../clientpositive/llap/acid_no_buckets.q.out | 351 ++++++--
.../clientpositive/llap/acid_subquery.q.out | 11 +
.../clientpositive/llap/acid_vectorization.q.out | 8 +
.../llap/acid_vectorization_original.q.out | 115 ++-
.../clientpositive/llap/acid_view_delete.q.out | 14 +-
.../clientpositive/llap/authorization_update.q.out | 4 +
.../llap/authorization_update_own_table.q.out | 4 +
.../llap/bucket_num_reducers_acid.q.out | 1 +
.../llap/bucket_num_reducers_acid2.q.out | 4 +
.../clientpositive/llap/check_constraint.q.out | 183 ++++-
.../llap/constant_prop_timestamp_date_cast.q.out | 69 +-
.../llap/create_transactional_full_acid.q.out | 8 +
.../llap/dynpart_sort_optimization_acid.q.out | 882 +++++++++++++++++----
.../llap/enforce_constraint_notnull.q.out | 204 ++++-
.../clientpositive/llap/explain_locks.q.out | 20 +-
.../explainanalyze_acid_with_direct_insert.q.out | 250 ++++--
.../clientpositive/llap/fetch_deleted_rows.q.out | 4 +
.../llap/fetch_deleted_rows_vector.q.out | 4 +
.../llap/insert_into_default_keyword.q.out | 190 ++++-
.../llap/insert_into_default_keyword_2.q.out | 85 +-
.../clientpositive/llap/insert_update_delete.q.out | 4 +
.../results/clientpositive/llap/llap_acid.q.out | 88 +-
.../results/clientpositive/llap/llap_acid2.q.out | 18 +
.../clientpositive/llap/llap_acid_fast.q.out | 88 +-
.../llap/masking_acid_no_masking.q.out | 4 +
.../llap/materialized_view_create_rewrite_4.q.out | 6 +
.../llap/materialized_view_create_rewrite_5.q.out | 6 +
.../llap/materialized_view_create_rewrite_8.q.out | 6 +
.../llap/schema_evol_orc_acid_part_update.q.out | 16 +-
.../schema_evol_orc_acid_part_update_llap_io.q.out | 16 +-
.../llap/schema_evol_orc_acid_table_update.q.out | 9 +-
...schema_evol_orc_acid_table_update_llap_io.q.out | 9 +-
.../llap/schema_evol_orc_acidvec_part_update.q.out | 16 +-
...hema_evol_orc_acidvec_part_update_llap_io.q.out | 16 +-
.../schema_evol_orc_acidvec_table_update.q.out | 9 +-
...ema_evol_orc_acidvec_table_update_llap_io.q.out | 9 +-
.../results/clientpositive/llap/sort_acid.q.out | 65 +-
.../results/clientpositive/llap/split_update.q.out | 225 ++++++
.../results/clientpositive/llap/stats_part2.q.out | 13 +-
.../llap/update_after_multiple_inserts.q.out | 5 +
...after_multiple_inserts_special_characters.q.out | 5 +
.../llap/update_all_non_partitioned.q.out | 4 +
.../llap/update_all_partitioned.q.out | 8 +
.../clientpositive/llap/update_all_types.q.out | 32 +
.../clientpositive/llap/update_tmp_table.q.out | 4 +
.../clientpositive/llap/update_two_cols.q.out | 5 +
.../llap/update_where_no_match.q.out | 4 +
.../llap/update_where_non_partitioned.q.out | 4 +
.../llap/update_where_partitioned.q.out | 5 +
.../results/clientpositive/llap/vector_acid4.q.out | 94 ++-
.../llap/vector_mapjoin_complex_values.q.out | 166 +++-
.../tez/acid_vectorization_original_tez.q.out | 127 ++-
.../clientpositive/tez/explainanalyze_5.q.out | 51 +-
.../clientpositive/tez/update_orig_table.q.out | 14 +
.../org/apache/hive/streaming/TestStreaming.java | 6 +-
82 files changed, 3662 insertions(+), 998 deletions(-)
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index 68564afa638..e6838106fdf 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -3381,11 +3381,22 @@ public class HiveConf extends Configuration {
MERGE_CARDINALITY_VIOLATION_CHECK("hive.merge.cardinality.check", true,
"Set to true to ensure that each SQL Merge statement ensures that for each row in the target\n" +
"table there is at most 1 matching row in the source table per SQL Specification."),
+ SPLIT_UPDATE("hive.split.update", true,
+ "If true, SQL Update statement will be rewritten to a multi-insert with 2 branches:\n" +
+ "representing delete of existing row and an insert of the new version of the row.\n" +
+ "Similarly Merge statement will handle WHEN MATCHED UPDATE by splitting it into 2\n" +
+ "branches of a multi-insert. Updating bucketing and partitioning columns should\n" +
+ "only be permitted if this is true."),
+ /**
+ * @deprecated Use {@link ConfVars#SPLIT_UPDATE} instead.
+ */
+ @Deprecated
MERGE_SPLIT_UPDATE("hive.merge.split.update", true,
"If true, SQL Merge statement will handle WHEN MATCHED UPDATE by splitting it into 2\n" +
"branches of a multi-insert, representing delete of existing row and an insert of\n" +
"the new version of the row. Updating bucketing and partitioning columns should\n" +
- "only be permitted if this is true."),
+ "only be permitted if this is true.\n" +
+ "Deprecated, use hive.split.update instead."),
OPTIMIZE_ACID_META_COLUMNS("hive.optimize.acid.meta.columns", true,
"If true, don't decode Acid metadata columns from storage unless" +
" they are needed."),
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestAcidOnTez.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestAcidOnTez.java
index 782794a8f67..e9f204db292 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestAcidOnTez.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestAcidOnTez.java
@@ -503,7 +503,7 @@ public class TestAcidOnTez {
{"{\"writeid\":1,\"bucketid\":536870913,\"rowid\":1}\t3\t4", "/delta_0000001_0000001_0001/bucket_00000_0"},
{"{\"writeid\":1,\"bucketid\":536870913,\"rowid\":2}\t1\t2", "/delta_0000001_0000001_0001/bucket_00000_0"},
{"{\"writeid\":1,\"bucketid\":536870914,\"rowid\":0}\t9\t10", "/delta_0000001_0000001_0002/bucket_00000_0"},
- {"{\"writeid\":2,\"bucketid\":536870912,\"rowid\":0}\t70\t80", "/delta_0000002_0000002_0000/bucket_00000_0"}
+ {"{\"writeid\":2,\"bucketid\":536870913,\"rowid\":0}\t70\t80", "/delta_0000002_0000002_0001/bucket_00000_0"}
};
Assert.assertEquals("Unexpected row count after update", expected2.length, rs.size());
//verify data and layout
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java
index c26e8321c4f..00c17212e2a 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java
@@ -227,6 +227,7 @@ public class TestJdbcDriver2 {
stmt.execute("set " + ConfVars.HIVE_SUPPORT_CONCURRENCY.varname + "=true");
stmt.execute("set " + ConfVars.HIVE_TXN_MANAGER.varname +
"=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager");
+ stmt.execute("drop table if exists transactional_crud");
stmt.execute("create table transactional_crud (a int, b int) stored as orc " +
"tblproperties('transactional'='true', 'transactional_properties'='default')");
int count = stmt.executeUpdate("insert into transactional_crud values(1,2),(3,4),(5,6)");
@@ -251,6 +252,44 @@ public class TestJdbcDriver2 {
assertEquals("1 row PreparedStatement update", 1, count);
}
+ @Test
+ public void testExceucteMergeCounts() throws Exception {
+ testExceucteMergeCounts(true);
+ }
+
+ @Test
+ public void testExceucteMergeCountsNoSplitUpdate() throws Exception {
+ testExceucteMergeCounts(false);
+ }
+
+ private void testExceucteMergeCounts(boolean splitUpdateEarly) throws Exception {
+
+ Statement stmt = con.createStatement();
+ stmt.execute("set " + ConfVars.SPLIT_UPDATE.varname + "=" + splitUpdateEarly);
+ stmt.execute("set " + ConfVars.HIVE_SUPPORT_CONCURRENCY.varname + "=true");
+ stmt.execute("set " + ConfVars.HIVE_TXN_MANAGER.varname +
+ "=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager");
+
+ stmt.execute("drop table if exists transactional_crud");
+ stmt.execute("drop table if exists source");
+
+ stmt.execute("create table transactional_crud (a int, b int) stored as orc " +
+ "tblproperties('transactional'='true', 'transactional_properties'='default')");
+ stmt.executeUpdate("insert into transactional_crud values(1,2),(3,4),(5,6),(7,8),(9,10)");
+
+ stmt.execute("create table source (a int, b int) stored as orc " +
+ "tblproperties('transactional'='true', 'transactional_properties'='default')");
+ stmt.executeUpdate("insert into source values(1,12),(3,14),(9,19),(100,100)");
+
+ int count = stmt.executeUpdate(" MERGE INTO transactional_crud as t using source as s ON t.a = s.a\n" +
+ " WHEN MATCHED AND s.a > 7 THEN DELETE\n" +
+ " WHEN MATCHED AND s.a <= 8 THEN UPDATE set b = 100\n" +
+ " WHEN NOT MATCHED THEN INSERT VALUES (s.a, s.b)");
+ stmt.close();
+
+ assertEquals("Statement merge", 4, count);
+ }
+
@AfterClass
public static void tearDownAfterClass() throws Exception {
Statement stmt = con.createStatement();
@@ -2855,6 +2894,7 @@ public class TestJdbcDriver2 {
stmt.execute("set hive.metastore.dml.events = true");
stmt.execute("set hive.repl.cm.enabled = true");
stmt.execute("set hive.repl.cmrootdir = cmroot");
+ stmt.execute("set " + ConfVars.HIVE_TXN_MANAGER.varname + "=" + ConfVars.HIVE_TXN_MANAGER.defaultStrVal);
stmt.execute("create database " + primaryDb + " with dbproperties('repl.source.for'='1,2,3')");
stmt.execute("create table " + primaryTblName + " (id int)");
stmt.execute("insert into " + primaryTblName + " values (1), (2)");
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/Context.java b/ql/src/java/org/apache/hadoop/hive/ql/Context.java
index f7c7c67a614..af16b2f0e07 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/Context.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/Context.java
@@ -19,7 +19,6 @@
package org.apache.hadoop.hive.ql;
import java.io.DataInput;
-import java.io.FileNotFoundException;
import java.io.IOException;
import java.net.URI;
import java.text.SimpleDateFormat;
@@ -165,6 +164,7 @@ public class Context {
* different prefixes to encode the purpose of different Insert branches
*/
private Map<Integer, DestClausePrefix> insertBranchToNamePrefix = new HashMap<>();
+ private int deleteBranchOfUpdateIdx = -1;
private Operation operation = Operation.OTHER;
private WmContext wmContext;
@@ -292,39 +292,47 @@ public class Context {
case OTHER:
return DestClausePrefix.INSERT;
case UPDATE:
+ if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.SPLIT_UPDATE)) {
+ return getMergeDestClausePrefix(curNode);
+ }
return DestClausePrefix.UPDATE;
case DELETE:
return DestClausePrefix.DELETE;
case MERGE:
- /* This is the structure expected here
- HiveParser.TOK_QUERY;
- HiveParser.TOK_FROM
- HiveParser.TOK_INSERT;
- HiveParser.TOK_INSERT_INTO;
- HiveParser.TOK_INSERT;
- HiveParser.TOK_INSERT_INTO;
- .....*/
- ASTNode insert = (ASTNode) curNode.getParent();
- assert insert != null && insert.getType() == HiveParser.TOK_INSERT;
- ASTNode query = (ASTNode) insert.getParent();
- assert query != null && query.getType() == HiveParser.TOK_QUERY;
-
- for(int childIdx = 1; childIdx < query.getChildCount(); childIdx++) {//1st child is TOK_FROM
- assert query.getChild(childIdx).getType() == HiveParser.TOK_INSERT;
- if(insert == query.getChild(childIdx)) {
- DestClausePrefix prefix = insertBranchToNamePrefix.get(childIdx);
- if(prefix == null) {
- throw new IllegalStateException("Found a node w/o branch mapping: '" +
- getMatchedText(insert) + "'");
- }
- return prefix;
- }
- }
- throw new IllegalStateException("Could not locate '" + getMatchedText(insert) + "'");
+ return getMergeDestClausePrefix(curNode);
default:
throw new IllegalStateException("Unexpected operation: " + operation);
}
}
+
+ private DestClausePrefix getMergeDestClausePrefix(ASTNode curNode) {
+ /* This is the structure expected here
+ HiveParser.TOK_QUERY;
+ HiveParser.TOK_FROM
+ HiveParser.TOK_INSERT;
+ HiveParser.TOK_INSERT_INTO;
+ HiveParser.TOK_INSERT;
+ HiveParser.TOK_INSERT_INTO;
+ .....*/
+ ASTNode insert = (ASTNode) curNode.getParent();
+ assert insert != null && insert.getType() == HiveParser.TOK_INSERT;
+ ASTNode query = (ASTNode) insert.getParent();
+ assert query != null && query.getType() == HiveParser.TOK_QUERY;
+
+ for(int childIdx = 1; childIdx < query.getChildCount(); childIdx++) {//1st child is TOK_FROM
+ assert query.getChild(childIdx).getType() == HiveParser.TOK_INSERT;
+ if(insert == query.getChild(childIdx)) {
+ DestClausePrefix prefix = insertBranchToNamePrefix.get(childIdx);
+ if(prefix == null) {
+ throw new IllegalStateException("Found a node w/o branch mapping: '" +
+ getMatchedText(insert) + "'");
+ }
+ return prefix;
+ }
+ }
+ throw new IllegalStateException("Could not locate '" + getMatchedText(insert) + "'");
+ }
+
/**
* Will make SemanticAnalyzer.Phase1Ctx#dest in subtree rooted at 'tree' use 'prefix'. This to
* handle multi-insert stmt that represents Merge stmt and has insert branches representing
@@ -336,6 +344,12 @@ public class Context {
return insertBranchToNamePrefix.put(pos, prefix);
}
+ public DestClausePrefix addDeleteOfUpdateDestNamePrefix(int pos, DestClausePrefix prefix) {
+ DestClausePrefix destClausePrefix = addDestNamePrefix(pos, prefix);
+ deleteBranchOfUpdateIdx = pos;
+ return destClausePrefix;
+ }
+
public Context(Configuration conf) {
this(conf, generateExecutionId());
}
@@ -1314,4 +1328,23 @@ public class Context {
public List<Pair<String, String>> getParsedTables() {
return parsedTables;
}
+
+ public boolean isDeleteBranchOfUpdate(String dest) {
+ if (!HiveConf.getBoolVar(conf, HiveConf.ConfVars.SPLIT_UPDATE) &&
+ !HiveConf.getBoolVar(conf, HiveConf.ConfVars.MERGE_SPLIT_UPDATE)) {
+ return false;
+ }
+
+ if (deleteBranchOfUpdateIdx > 0) {
+ return dest.endsWith(Integer.toString(deleteBranchOfUpdateIdx - 1));
+ }
+
+ for (Context subContext : subContexts) {
+ if (subContext.isDeleteBranchOfUpdate(dest)) {
+ return true;
+ }
+ }
+
+ return false;
+ }
}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
index 4204d920f18..0d95b62e292 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
@@ -1145,7 +1145,8 @@ public class FileSinkOperator extends TerminalOperator<FileSinkDesc> implements
cntr = 0;
numRows = 1;
}
- LOG.info(toString() + ": records written - " + numRows);
+ LOG.info("{}: {} written - {}",
+ this, conf.isDeleteOfSplitUpdate() ? "delete delta records" : "records", numRows);
}
int writerOffset;
@@ -1444,8 +1445,10 @@ public class FileSinkOperator extends TerminalOperator<FileSinkDesc> implements
@Override
public void closeOp(boolean abort) throws HiveException {
- row_count.set(numRows);
- LOG.info(toString() + ": records written - " + numRows);
+ row_count.set(conf.isDeleteOfSplitUpdate() ? 0 : numRows);
+
+ LOG.info("{}: {} written - {}",
+ this, conf.isDeleteOfSplitUpdate() ? "delete delta records" : "records", numRows);
if (!bDynParts && !filesCreated) {
boolean isTez = "tez".equalsIgnoreCase(
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/AcidExportSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/AcidExportSemanticAnalyzer.java
index 2b834361019..301bbbdddbf 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/AcidExportSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/AcidExportSemanticAnalyzer.java
@@ -63,12 +63,19 @@ public class AcidExportSemanticAnalyzer extends RewriteSemanticAnalyzer {
super(queryState);
}
- protected void analyze(ASTNode tree) throws SemanticException {
+ @Override
+ protected ASTNode getTargetTableNode(ASTNode tree) {
+ ASTNode tableTree = (ASTNode)tree.getChild(0);
+ assert tableTree != null && tableTree.getType() == HiveParser.TOK_TAB;
+ return (ASTNode) tableTree.getChild(0);
+ }
+
+ protected void analyze(ASTNode tree, Table table, ASTNode tableNameNode) throws SemanticException {
if (tree.getToken().getType() != HiveParser.TOK_EXPORT) {
throw new RuntimeException("Asked to parse token " + tree.getName() + " in " +
"AcidExportSemanticAnalyzer");
}
- analyzeAcidExport(tree);
+ analyzeAcidExport(tree, table, tableNameNode);
}
/**
@@ -80,7 +87,7 @@ public class AcidExportSemanticAnalyzer extends RewriteSemanticAnalyzer {
* insert into tmpTable select * from <export table> to filter/apply the events in current
* context and then export the tmpTable. This export artifact can now be imported into any
* table on any cluster (subject to schema checks etc).
- * See {@link #analyzeAcidExport(ASTNode)}
+ * See {@link #analyzeAcidExport(ASTNode, Table , ASTNode)}
* @param tree Export statement
* @return true if exporting an Acid table.
*/
@@ -115,13 +122,9 @@ public class AcidExportSemanticAnalyzer extends RewriteSemanticAnalyzer {
* Using a true temp (session level) table means it should not affect replication and the table
* is not visible outside the Session that created for security
*/
- private void analyzeAcidExport(ASTNode ast) throws SemanticException {
+ private void analyzeAcidExport(ASTNode ast, Table exportTable, ASTNode tokRefOrNameExportTable) throws SemanticException {
assert ast != null && ast.getToken() != null && ast.getToken().getType() == HiveParser.TOK_EXPORT;
- ASTNode tableTree = (ASTNode)ast.getChild(0);
- assert tableTree != null && tableTree.getType() == HiveParser.TOK_TAB;
- ASTNode tokRefOrNameExportTable = (ASTNode) tableTree.getChild(0);
- Table exportTable = getTargetTable(tokRefOrNameExportTable);
-
+
if (exportTable != null && (exportTable.isView() || exportTable.isMaterializedView())) {
throw new SemanticException("Views and Materialized Views can not be exported.");
}
@@ -169,19 +172,14 @@ public class AcidExportSemanticAnalyzer extends RewriteSemanticAnalyzer {
//now generate insert statement
//insert into newTableName select * from ts <where partition spec>
- StringBuilder rewrittenQueryStr = generateExportQuery(newTable.getPartCols(), tokRefOrNameExportTable, tableTree,
- newTableName);
+ StringBuilder rewrittenQueryStr = generateExportQuery(
+ newTable.getPartCols(), tokRefOrNameExportTable, (ASTNode) tokRefOrNameExportTable.parent, newTableName);
ReparseResult rr = parseRewrittenQuery(rewrittenQueryStr, ctx.getCmd());
Context rewrittenCtx = rr.rewrittenCtx;
rewrittenCtx.setIsUpdateDeleteMerge(false); //it's set in parseRewrittenQuery()
ASTNode rewrittenTree = rr.rewrittenTree;
- try {
- useSuper = true;
- //newTable has to exist at this point to compile
- super.analyze(rewrittenTree, rewrittenCtx);
- } finally {
- useSuper = false;
- }
+ //newTable has to exist at this point to compile
+ analyzeRewrittenTree(rewrittenTree, rewrittenCtx);
//now we have the rootTasks set up for Insert ... Select
removeStatsTasks(rootTasks);
//now make an ExportTask from temp table
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 03233ca5978..72dda4154dd 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
@@ -18,6 +18,7 @@
package org.apache.hadoop.hive.ql.parse;
import java.util.ArrayList;
+import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.IdentityHashMap;
@@ -53,68 +54,18 @@ public class MergeSemanticAnalyzer extends RewriteSemanticAnalyzer {
}
@Override
- public void analyze(ASTNode tree) throws SemanticException {
+ protected ASTNode getTargetTableNode(ASTNode tree) {
+ return (ASTNode)tree.getChild(0);
+ }
+
+ @Override
+ public void analyze(ASTNode tree, Table targetTable, ASTNode tableNameNode) throws SemanticException {
if (tree.getToken().getType() != HiveParser.TOK_MERGE) {
throw new RuntimeException("Asked to parse token " + tree.getName() + " in " +
"MergeSemanticAnalyzer");
}
ctx.setOperation(Context.Operation.MERGE);
- analyzeMerge(tree);
- }
-
- private static final String INDENT = " ";
-
- private IdentifierQuoter quotedIdentifierHelper;
-
- /**
- * This allows us to take an arbitrary ASTNode and turn it back into SQL that produced it.
- * Since HiveLexer.g is written such that it strips away any ` (back ticks) around
- * quoted identifiers we need to add those back to generated SQL.
- * Additionally, the parser only produces tokens of type Identifier and never
- * QuotedIdentifier (HIVE-6013). So here we just quote all identifiers.
- * (') around String literals are retained w/o issues
- */
- private static class IdentifierQuoter {
- private final TokenRewriteStream trs;
- private final IdentityHashMap<ASTNode, ASTNode> visitedNodes = new IdentityHashMap<>();
-
- IdentifierQuoter(TokenRewriteStream trs) {
- this.trs = trs;
- if (trs == null) {
- throw new IllegalArgumentException("Must have a TokenRewriteStream");
- }
- }
-
- private void visit(ASTNode n) {
- if (n.getType() == HiveParser.Identifier) {
- if (visitedNodes.containsKey(n)) {
- /**
- * Since we are modifying the stream, it's not idempotent. Ideally, the caller would take
- * care to only quote Identifiers in each subtree once, but this makes it safe
- */
- return;
- }
- visitedNodes.put(n, n);
- trs.insertBefore(n.getToken(), "`");
- trs.insertAfter(n.getToken(), "`");
- }
- if (n.getChildCount() <= 0) {
- return;
- }
- for (Node c : n.getChildren()) {
- visit((ASTNode)c);
- }
- }
- }
-
- /**
- * This allows us to take an arbitrary ASTNode and turn it back into SQL that produced it without
- * needing to understand what it is (except for QuotedIdentifiers).
- */
- private String getMatchedText(ASTNode n) {
- quotedIdentifierHelper.visit(n);
- return ctx.getTokenRewriteStream().toString(n.getTokenStartIndex(),
- n.getTokenStopIndex() + 1).trim();
+ analyzeMerge(tree, targetTable, tableNameNode);
}
/**
@@ -129,8 +80,7 @@ public class MergeSemanticAnalyzer extends RewriteSemanticAnalyzer {
*
* @throws SemanticException
*/
- private void analyzeMerge(ASTNode tree) throws SemanticException {
- quotedIdentifierHelper = new IdentifierQuoter(ctx.getTokenRewriteStream());
+ private void analyzeMerge(ASTNode tree, Table targetTable, ASTNode targetNameNode) throws SemanticException {
/*
* See org.apache.hadoop.hive.ql.parse.TestMergeStatement for some examples of the merge AST
For example, given:
@@ -167,9 +117,8 @@ public class MergeSemanticAnalyzer extends RewriteSemanticAnalyzer {
source is empty? This should be a runtime error - maybe not the outer side of ROJ is empty => the join produces 0
rows. If supporting WHEN NOT MATCHED BY SOURCE, then this should be a runtime error
*/
- ASTNode target = (ASTNode)tree.getChild(0);
ASTNode source = (ASTNode)tree.getChild(1);
- String targetName = getSimpleTableName(target);
+ String targetName = getSimpleTableName(targetNameNode);
String sourceName = getSimpleTableName(source);
ASTNode onClause = (ASTNode) tree.getChild(2);
String onClauseAsText = getMatchedText(onClause);
@@ -182,17 +131,11 @@ public class MergeSemanticAnalyzer extends RewriteSemanticAnalyzer {
hasHint = true;
whenClauseBegins++;
}
- Table targetTable = getTargetTable(target);
- validateTargetTable(targetTable);
List<ASTNode> whenClauses = findWhenClauses(tree, whenClauseBegins);
- StringBuilder rewrittenQueryStr = new StringBuilder("FROM\n");
+ StringBuilder rewrittenQueryStr = createRewrittenQueryStrBuilder();
+ appendTarget(rewrittenQueryStr, targetNameNode, targetName);
- rewrittenQueryStr.append(INDENT).append(getFullTableNameForSQL(target));
- if (isAliased(target)) {
- rewrittenQueryStr.append(" ").append(targetName);
- }
- rewrittenQueryStr.append('\n');
rewrittenQueryStr.append(INDENT).append(chooseJoinType(whenClauses)).append("\n");
if (source.getType() == HiveParser.TOK_SUBQUERY) {
//this includes the mandatory alias
@@ -211,8 +154,8 @@ public class MergeSemanticAnalyzer extends RewriteSemanticAnalyzer {
if (hasHint) {
hintStr = " /*+ " + qHint.getText() + " */ ";
}
- final boolean splitUpdateEarly =
- HiveConf.getBoolVar(conf, HiveConf.ConfVars.MERGE_SPLIT_UPDATE);
+ final boolean splitUpdateEarly = HiveConf.getBoolVar(conf, HiveConf.ConfVars.SPLIT_UPDATE) ||
+ HiveConf.getBoolVar(conf, HiveConf.ConfVars.MERGE_SPLIT_UPDATE);
/**
* 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>"
@@ -227,13 +170,13 @@ public class MergeSemanticAnalyzer extends RewriteSemanticAnalyzer {
switch (getWhenClauseOperation(whenClause).getType()) {
case HiveParser.TOK_INSERT:
numInsertClauses++;
- handleInsert(whenClause, rewrittenQueryStr, target, onClause,
+ handleInsert(whenClause, rewrittenQueryStr, targetNameNode, onClause,
targetTable, targetName, onClauseAsText, hintProcessed ? null : hintStr);
hintProcessed = true;
break;
case HiveParser.TOK_UPDATE:
numWhenMatchedUpdateClauses++;
- String s = handleUpdate(whenClause, rewrittenQueryStr, target,
+ String s = handleUpdate(whenClause, rewrittenQueryStr, targetNameNode,
onClauseAsText, targetTable, extraPredicate, hintProcessed ? null : hintStr,
splitUpdateEarly);
hintProcessed = true;
@@ -243,8 +186,8 @@ public class MergeSemanticAnalyzer extends RewriteSemanticAnalyzer {
break;
case HiveParser.TOK_DELETE:
numWhenMatchedDeleteClauses++;
- String s1 = handleDelete(whenClause, rewrittenQueryStr, target,
- onClauseAsText, targetTable, extraPredicate, hintProcessed ? null : hintStr, false);
+ String s1 = handleDelete(whenClause, rewrittenQueryStr, targetNameNode,
+ onClauseAsText, extraPredicate, hintProcessed ? null : hintStr, false);
hintProcessed = true;
if (numWhenMatchedUpdateClauses + numWhenMatchedDeleteClauses == 1) {
extraPredicate = s1; //i.e. it's the 1st WHEN MATCHED
@@ -266,7 +209,7 @@ public class MergeSemanticAnalyzer extends RewriteSemanticAnalyzer {
throw new SemanticException(ErrorMsg.MERGE_PREDIACTE_REQUIRED, ctx.getCmd());
}
- boolean validating = handleCardinalityViolation(rewrittenQueryStr, target, onClauseAsText, targetTable,
+ boolean validating = handleCardinalityViolation(rewrittenQueryStr, targetNameNode, onClauseAsText, targetTable,
numWhenMatchedDeleteClauses == 0 && numWhenMatchedUpdateClauses == 0);
ReparseResult rr = parseRewrittenQuery(rewrittenQueryStr, ctx.getCmd());
Context rewrittenCtx = rr.rewrittenCtx;
@@ -296,7 +239,7 @@ public class MergeSemanticAnalyzer extends RewriteSemanticAnalyzer {
Update of that partition since that update by definition cannot be of the same row.
If we ever enforce unique constraints we may have to have I+I be in conflict*/
rewrittenCtx.addDestNamePrefix(insClauseIdx, Context.DestClausePrefix.INSERT);
- rewrittenCtx.addDestNamePrefix(++insClauseIdx, Context.DestClausePrefix.DELETE);
+ rewrittenCtx.addDeleteOfUpdateDestNamePrefix(++insClauseIdx, Context.DestClausePrefix.DELETE);
}
break;
case HiveParser.TOK_DELETE:
@@ -311,12 +254,7 @@ public class MergeSemanticAnalyzer extends RewriteSemanticAnalyzer {
rewrittenCtx.addDestNamePrefix(rewrittenTree.getChildCount() - 1, Context.DestClausePrefix.INSERT);
}
- try {
- useSuper = true;
- super.analyze(rewrittenTree, rewrittenCtx);
- } finally {
- useSuper = false;
- }
+ analyzeRewrittenTree(rewrittenTree, rewrittenCtx);
updateOutputs(targetTable);
}
@@ -402,15 +340,9 @@ public class MergeSemanticAnalyzer extends RewriteSemanticAnalyzer {
assert whenMatchedUpdateClause.getType() == HiveParser.TOK_MATCHED;
assert getWhenClauseOperation(whenMatchedUpdateClause).getType() == HiveParser.TOK_UPDATE;
String targetName = getSimpleTableName(target);
- rewrittenQueryStr.append("INSERT INTO ").append(getFullTableNameForSQL(target));
- addPartitionColsToInsert(targetTable.getPartCols(), rewrittenQueryStr);
- rewrittenQueryStr.append(" -- update clause").append(splitUpdateEarly ? "(insert part)": "")
- .append("\n SELECT ");
- if (hintStr != null) {
- rewrittenQueryStr.append(hintStr);
- }
+ List<String> values = new ArrayList<>(targetTable.getCols().size() + (splitUpdateEarly ? 1 : 0));
if(!splitUpdateEarly) {
- rewrittenQueryStr.append(targetName).append(".ROW__ID, ");
+ values.add(targetName + ".ROW__ID, ");
}
ASTNode setClause = (ASTNode)getWhenClauseOperation(whenMatchedUpdateClause).getChild(0);
@@ -424,11 +356,7 @@ public class MergeSemanticAnalyzer extends RewriteSemanticAnalyzer {
//names
List<FieldSchema> nonPartCols = targetTable.getCols();
Map<String, String> colNameToDefaultConstraint = getColNameToDefaultValueMap(targetTable);
- for(int i = 0; i < nonPartCols.size(); i++) {
- FieldSchema fs = nonPartCols.get(i);
- if(i > 0) {
- rewrittenQueryStr.append(", ");
- }
+ for (FieldSchema fs : nonPartCols) {
String name = fs.getName();
if (setColsExprs.containsKey(name)) {
ASTNode setColExpr = setColsExprs.get(name);
@@ -444,23 +372,25 @@ public class MergeSemanticAnalyzer extends RewriteSemanticAnalyzer {
String rhsExp = getMatchedText(setColsExprs.get(name));
//"set a=5, b=8" - rhsExp picks up the next char (e.g. ',') from the token stream
switch (rhsExp.charAt(rhsExp.length() - 1)) {
- case ',':
- case '\n':
- rhsExp = rhsExp.substring(0, rhsExp.length() - 1);
- break;
- default:
- //do nothing
+ case ',':
+ case '\n':
+ rhsExp = rhsExp.substring(0, rhsExp.length() - 1);
+ break;
+ default:
+ //do nothing
}
- rewrittenQueryStr.append(rhsExp);
+ values.add(rhsExp);
} else {
- rewrittenQueryStr.append(getSimpleTableName(target))
- .append(".")
- .append(HiveUtils.unparseIdentifier(name, this.conf));
+ values.add(targetName + "." + HiveUtils.unparseIdentifier(name, this.conf));
}
}
- addPartitionColsToSelect(targetTable.getPartCols(), rewrittenQueryStr, target);
- rewrittenQueryStr.append("\n WHERE ").append(onClauseAsString);
+ addPartitionColsAsValues(targetTable.getPartCols(), targetName, values);
+
+ rewrittenQueryStr.append(" -- update clause").append(splitUpdateEarly ? " (insert part)": "").append("\n");
+ appendInsertBranch(rewrittenQueryStr, hintStr, values);
+
+ rewrittenQueryStr.append(INDENT).append("WHERE ").append(onClauseAsString);
String extraPredicate = getWhenClausePredicate(whenMatchedUpdateClause);
if (extraPredicate != null) {
//we have WHEN MATCHED AND <boolean expr> THEN DELETE
@@ -470,8 +400,7 @@ public class MergeSemanticAnalyzer extends RewriteSemanticAnalyzer {
rewrittenQueryStr.append(" AND NOT(").append(deleteExtraPredicate).append(")");
}
if(!splitUpdateEarly) {
- rewrittenQueryStr.append("\n SORT BY ");
- rewrittenQueryStr.append(targetName).append(".ROW__ID ");
+ appendSortBy(rewrittenQueryStr, Collections.singletonList(targetName + ".ROW__ID "));
}
rewrittenQueryStr.append("\n");
@@ -485,8 +414,9 @@ public class MergeSemanticAnalyzer extends RewriteSemanticAnalyzer {
* this is part of the WHEN MATCHED UPDATE, so we ignore any 'extra predicate' generated
* by this call to handleDelete()
*/
+ rewrittenQueryStr.append(" -- update clause (delete part)\n");
handleDelete(whenMatchedUpdateClause, rewrittenQueryStr, target, onClauseAsString,
- targetTable, deleteExtraPredicate, hintStr, true);
+ deleteExtraPredicate, hintStr, true);
}
return extraPredicate;
@@ -497,28 +427,17 @@ public class MergeSemanticAnalyzer extends RewriteSemanticAnalyzer {
* @param updateExtraPredicate - see notes at caller
*/
private String handleDelete(ASTNode whenMatchedDeleteClause, StringBuilder rewrittenQueryStr,
- ASTNode target, String onClauseAsString, Table targetTable, String updateExtraPredicate,
+ ASTNode target, String onClauseAsString, String updateExtraPredicate,
String hintStr, boolean splitUpdateEarly) throws SemanticException {
assert whenMatchedDeleteClause.getType() == HiveParser.TOK_MATCHED;
assert (splitUpdateEarly &&
getWhenClauseOperation(whenMatchedDeleteClause).getType() == HiveParser.TOK_UPDATE) ||
getWhenClauseOperation(whenMatchedDeleteClause).getType() == HiveParser.TOK_DELETE;
- List<FieldSchema> partCols = targetTable.getPartCols();
String targetName = getSimpleTableName(target);
- rewrittenQueryStr.append("INSERT INTO ").append(getFullTableNameForSQL(target));
- addPartitionColsToInsert(partCols, rewrittenQueryStr);
- if(splitUpdateEarly) {
- rewrittenQueryStr.append(" -- update clause (delete part)\n SELECT ");
- } else {
- rewrittenQueryStr.append(" -- delete clause\n SELECT ");
- }
- if (hintStr != null) {
- rewrittenQueryStr.append(hintStr);
- }
- rewrittenQueryStr.append(targetName).append(".ROW__ID ");
- addPartitionColsToSelect(partCols, rewrittenQueryStr, target);
- rewrittenQueryStr.append("\n WHERE ").append(onClauseAsString);
+ appendDeleteBranch(rewrittenQueryStr, hintStr, targetName, Collections.singletonList(targetName + ".ROW__ID"));
+
+ rewrittenQueryStr.append(INDENT).append("WHERE ").append(onClauseAsString);
String extraPredicate = getWhenClausePredicate(whenMatchedDeleteClause);
if (extraPredicate != null) {
//we have WHEN MATCHED AND <boolean expr> THEN DELETE
@@ -527,8 +446,7 @@ public class MergeSemanticAnalyzer extends RewriteSemanticAnalyzer {
if (updateExtraPredicate != null) {
rewrittenQueryStr.append(" AND NOT(").append(updateExtraPredicate).append(")");
}
- rewrittenQueryStr.append("\n SORT BY ");
- rewrittenQueryStr.append(targetName).append(".ROW__ID \n");
+ appendSortBy(rewrittenQueryStr, Collections.singletonList(targetName + ".ROW__ID "));
return extraPredicate;
}
@@ -536,20 +454,6 @@ public class MergeSemanticAnalyzer extends RewriteSemanticAnalyzer {
return " at " + ASTErrorUtils.renderPosition(n);
}
- private boolean isAliased(ASTNode n) {
- switch (n.getType()) {
- case HiveParser.TOK_TABREF:
- return findTabRefIdxs(n)[0] != 0;
- case HiveParser.TOK_TABNAME:
- return false;
- case HiveParser.TOK_SUBQUERY:
- assert n.getChildCount() > 1 : "Expected Derived Table to be aliased";
- return true;
- default:
- throw raiseWrongType("TOK_TABREF|TOK_TABNAME", n);
- }
- }
-
/**
* Collect WHEN clauses from Merge statement AST.
*/
@@ -797,4 +701,9 @@ public class MergeSemanticAnalyzer extends RewriteSemanticAnalyzer {
return sb.toString();
}
}
+
+ @Override
+ protected boolean allowOutputMultipleTimes() {
+ return conf.getBoolVar(HiveConf.ConfVars.SPLIT_UPDATE) || conf.getBoolVar(HiveConf.ConfVars.MERGE_SPLIT_UPDATE);
+ }
}
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 6e915829982..cd7de19b120 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
@@ -17,13 +17,15 @@
*/
package org.apache.hadoop.hive.ql.parse;
-import java.io.IOException;
import java.util.ArrayList;
+import java.util.IdentityHashMap;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
+import org.antlr.runtime.TokenRewriteStream;
+import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.hive.common.TableName;
import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
@@ -35,6 +37,7 @@ import org.apache.hadoop.hive.ql.QueryState;
import org.apache.hadoop.hive.ql.hooks.Entity;
import org.apache.hadoop.hive.ql.hooks.ReadEntity;
import org.apache.hadoop.hive.ql.hooks.WriteEntity;
+import org.apache.hadoop.hive.ql.io.AcidUtils;
import org.apache.hadoop.hive.ql.lib.Node;
import org.apache.hadoop.hive.ql.metadata.Hive;
import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -45,6 +48,8 @@ import org.apache.hadoop.hive.ql.metadata.VirtualColumn;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import static org.apache.commons.lang3.StringUtils.isNotBlank;
+
/**
* A subclass of the {@link org.apache.hadoop.hive.ql.parse.SemanticAnalyzer} that just handles
@@ -56,6 +61,11 @@ public abstract class RewriteSemanticAnalyzer extends CalcitePlanner {
protected static final Logger LOG = LoggerFactory.getLogger(RewriteSemanticAnalyzer.class);
protected boolean useSuper = false;
+ protected static final String INDENT = " ";
+ private IdentifierQuoter quotedIdentifierHelper;
+ private Table targetTable;
+ private String targetTableFullName;
+
RewriteSemanticAnalyzer(QueryState queryState) throws SemanticException {
super(queryState);
@@ -66,12 +76,52 @@ public abstract class RewriteSemanticAnalyzer extends CalcitePlanner {
if (useSuper) {
super.analyzeInternal(tree);
} else {
+ quotedIdentifierHelper = new IdentifierQuoter(ctx.getTokenRewriteStream());
analyze(tree);
cleanUpMetaColumnAccessControl();
}
}
- protected abstract void analyze(ASTNode tree) throws SemanticException;
+ protected abstract ASTNode getTargetTableNode(ASTNode tree);
+
+ private void analyze(ASTNode tree) throws SemanticException {
+ ASTNode tableName = getTargetTableNode(tree);
+
+ targetTableFullName = getFullTableNameForSQL(tableName);
+ targetTable = getTable(tableName, db, true);
+ validateTxnManager(targetTable);
+ validateTargetTable(targetTable);
+ analyze(tree, targetTable, tableName);
+ }
+
+ protected abstract void analyze(ASTNode tree, Table table, ASTNode tableName) throws SemanticException;
+
+ public void analyzeRewrittenTree(ASTNode rewrittenTree, Context rewrittenCtx) throws SemanticException {
+ try {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Rewritten AST {}", rewrittenTree.dump());
+ }
+ useSuper = true;
+ super.analyze(rewrittenTree, rewrittenCtx);
+ } finally {
+ useSuper = false;
+ }
+ }
+
+ /**
+ * 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)
+ throws SemanticException {
+ // If the table is partitioned, we need to select the partition columns as well.
+ if (partCols != null) {
+ for (FieldSchema fschema : partCols) {
+ rewrittenQueryStr.append(", ");
+ rewrittenQueryStr.append(HiveUtils.unparseIdentifier(fschema.getName(), this.conf));
+ }
+ }
+ }
/**
* Append list of partition columns to Insert statement, i.e. the 2nd set of partCol1,partCol2
@@ -80,21 +130,33 @@ public abstract class RewriteSemanticAnalyzer extends CalcitePlanner {
*/
protected void addPartitionColsToSelect(List<FieldSchema> partCols, StringBuilder rewrittenQueryStr,
ASTNode target) throws SemanticException {
- String targetName = target != null ? getSimpleTableName(target) : null;
+ addPartitionColsToSelect(partCols, 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
+ */
+ protected void addPartitionColsToSelect(List<FieldSchema> partCols, 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) {
rewrittenQueryStr.append(", ");
- //would be nice if there was a way to determine if quotes are needed
- if (targetName != null) {
- rewrittenQueryStr.append(targetName).append('.');
- }
+ rewrittenQueryStr.append(alias).append('.');
rewrittenQueryStr.append(HiveUtils.unparseIdentifier(fschema.getName(), this.conf));
}
}
}
+ protected void addPartitionColsAsValues(List<FieldSchema> partCols, String alias, List<String> values) {
+ if (partCols == null) {
+ return;
+ }
+ partCols.forEach(
+ fieldSchema -> values.add(alias + "." + HiveUtils.unparseIdentifier(fieldSchema.getName(), this.conf)));
+ }
+
/**
* Assert that we are not asked to update a bucketing column or partition column.
* @param colName it's the A in "SET A = B"
@@ -164,13 +226,6 @@ public abstract class RewriteSemanticAnalyzer extends CalcitePlanner {
return setCols;
}
- /**
- * @return the Metastore representation of the target table
- */
- protected Table getTargetTable(ASTNode tabRef) throws SemanticException {
- return getTable(tabRef, db, true);
- }
-
/**
* @param throwException if false, return null if table doesn't exist, else throw
*/
@@ -276,6 +331,12 @@ public abstract class RewriteSemanticAnalyzer extends CalcitePlanner {
return new ReparseResult(rewrittenTree, rewrittenCtx);
}
+ private void validateTxnManager(Table mTable) throws SemanticException {
+ if (!AcidUtils.acidTableWithoutTransactions(mTable) && !getTxnMgr().supportsAcid()) {
+ throw new SemanticException(ErrorMsg.ACID_OP_ON_NONACID_TXNMGR.getMsg());
+ }
+ }
+
/**
* Assert it supports Acid write.
*/
@@ -423,4 +484,130 @@ public abstract class RewriteSemanticAnalyzer extends CalcitePlanner {
rewrittenCtx = c;
}
}
+
+ // Patch up the projection list for updates, putting back the original set expressions.
+ // Walk through the projection list and replace the column names with the
+ // expressions from the original update. Under the TOK_SELECT (see above) the structure
+ // looks like:
+ // TOK_SELECT -> TOK_SELEXPR -> expr
+ // \-> TOK_SELEXPR -> expr ...
+ protected void patchProjectionForUpdate(ASTNode insertBranch, Map<Integer, ASTNode> setColExprs) {
+ ASTNode rewrittenSelect = (ASTNode) insertBranch.getChildren().get(1);
+ assert rewrittenSelect.getToken().getType() == HiveParser.TOK_SELECT :
+ "Expected TOK_SELECT as second child of TOK_INSERT but found " + rewrittenSelect.getName();
+ for (Map.Entry<Integer, ASTNode> entry : setColExprs.entrySet()) {
+ ASTNode selExpr = (ASTNode) rewrittenSelect.getChildren().get(entry.getKey());
+ assert selExpr.getToken().getType() == HiveParser.TOK_SELEXPR :
+ "Expected child of TOK_SELECT to be TOK_SELEXPR but was " + selExpr.getName();
+ // Now, change it's child
+ selExpr.setChild(0, entry.getValue());
+ }
+ }
+
+ protected StringBuilder createRewrittenQueryStrBuilder() {
+ return new StringBuilder("FROM\n");
+ }
+
+ protected void appendTarget(StringBuilder rewrittenQueryStr, ASTNode target, String targetName) {
+ rewrittenQueryStr.append(INDENT).append(targetTableFullName);
+ if (isAliased(target)) {
+ rewrittenQueryStr.append(" ").append(targetName);
+ }
+ rewrittenQueryStr.append('\n');
+ }
+
+ protected boolean isAliased(ASTNode n) {
+ switch (n.getType()) {
+ case HiveParser.TOK_TABREF:
+ return findTabRefIdxs(n)[0] != 0;
+ case HiveParser.TOK_TABNAME:
+ return false;
+ case HiveParser.TOK_SUBQUERY:
+ assert n.getChildCount() > 1 : "Expected Derived Table to be aliased";
+ return true;
+ default:
+ throw raiseWrongType("TOK_TABREF|TOK_TABNAME", n);
+ }
+ }
+
+ protected void appendInsertBranch(StringBuilder rewrittenQueryStr, String hintStr, List<String> values) {
+ rewrittenQueryStr.append("INSERT INTO ").append(targetTableFullName);
+ addPartitionColsToInsert(targetTable.getPartCols(), rewrittenQueryStr);
+ rewrittenQueryStr.append("\n");
+
+ rewrittenQueryStr.append(INDENT);
+ rewrittenQueryStr.append("SELECT ");
+ if (isNotBlank(hintStr)) {
+ rewrittenQueryStr.append(hintStr);
+ }
+
+ rewrittenQueryStr.append(StringUtils.join(values, ","));
+ rewrittenQueryStr.append("\n");
+ }
+
+ protected void appendDeleteBranch(
+ StringBuilder rewrittenQueryStr, String hintStr, String alias, List<String> values) {
+ List<String> deleteValues = new ArrayList<>(targetTable.getPartCols().size() + values.size());
+ deleteValues.addAll(values);
+ addPartitionColsAsValues(targetTable.getPartCols(), alias, deleteValues);
+
+ appendInsertBranch(rewrittenQueryStr, hintStr, deleteValues);
+ }
+
+ protected void appendSortBy(StringBuilder rewrittenQueryStr, List<String> keys) {
+ rewrittenQueryStr.append(INDENT).append("SORT BY ");
+ rewrittenQueryStr.append(StringUtils.join(keys, ","));
+ rewrittenQueryStr.append("\n");
+ }
+
+ /**
+ * This allows us to take an arbitrary ASTNode and turn it back into SQL that produced it.
+ * Since HiveLexer.g is written such that it strips away any ` (back ticks) around
+ * quoted identifiers we need to add those back to generated SQL.
+ * Additionally, the parser only produces tokens of type Identifier and never
+ * QuotedIdentifier (HIVE-6013). So here we just quote all identifiers.
+ * (') around String literals are retained w/o issues
+ */
+ private static class IdentifierQuoter {
+ private final TokenRewriteStream trs;
+ private final IdentityHashMap<ASTNode, ASTNode> visitedNodes = new IdentityHashMap<>();
+
+ IdentifierQuoter(TokenRewriteStream trs) {
+ this.trs = trs;
+ if (trs == null) {
+ throw new IllegalArgumentException("Must have a TokenRewriteStream");
+ }
+ }
+
+ private void visit(ASTNode n) {
+ if (n.getType() == HiveParser.Identifier) {
+ if (visitedNodes.containsKey(n)) {
+ /**
+ * Since we are modifying the stream, it's not idempotent. Ideally, the caller would take
+ * care to only quote Identifiers in each subtree once, but this makes it safe
+ */
+ return;
+ }
+ visitedNodes.put(n, n);
+ trs.insertBefore(n.getToken(), "`");
+ trs.insertAfter(n.getToken(), "`");
+ }
+ if (n.getChildCount() <= 0) {
+ return;
+ }
+ for (Node c : n.getChildren()) {
+ visit((ASTNode)c);
+ }
+ }
+ }
+
+ /**
+ * This allows us to take an arbitrary ASTNode and turn it back into SQL that produced it without
+ * needing to understand what it is (except for QuotedIdentifiers).
+ */
+ protected String getMatchedText(ASTNode n) {
+ quotedIdentifierHelper.visit(n);
+ return ctx.getTokenRewriteStream().toString(n.getTokenStartIndex(),
+ n.getTokenStopIndex() + 1).trim();
+ }
}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
index 881f265081d..2e8c4986206 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
@@ -7960,7 +7960,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
&& destinationTable != null
&& (!destinationTable.isNonNative() || destinationTable.getStorageHandler().commitInMoveTask())
&& !destTableIsTemporary && !destTableIsMaterialization
- && ColumnStatsAutoGatherContext.canRunAutogatherStats(fso)) {
+ && ColumnStatsAutoGatherContext.canRunAutogatherStats(fso)
+ && !(this instanceof UpdateDeleteSemanticAnalyzer)) {
if (destType == QBMetaData.DEST_TABLE) {
genAutoColumnStatsGatheringPipeline(destinationTable, partSpec, input,
qb.getParseInfo().isInsertIntoTable(destinationTable.getDbName(), destinationTable.getTableName()),
@@ -8229,7 +8230,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
conf.getBoolVar(HiveConf.ConfVars.COMPRESSRESULT), currentTableId, rsCtx.isMultiFileSpray(),
canBeMerged, rsCtx.getNumFiles(), rsCtx.getTotalFiles(), rsCtx.getPartnCols(), dpCtx,
dest_path, mmWriteId, isMmCtas, isInsertOverwrite, qb.getIsQuery(),
- qb.isCTAS() || qb.isMaterializedView(), isDirectInsert, acidOperation);
+ qb.isCTAS() || qb.isMaterializedView(), isDirectInsert, acidOperation,
+ ctx.isDeleteBranchOfUpdate(dest));
fileSinkDesc.setMoveTaskId(moveTaskId);
boolean isHiveServerQuery = SessionState.get().isHiveServerQuery();
@@ -8369,8 +8371,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
if ((dpCtx == null || dpCtx.getNumDPCols() == 0)) {
output = new WriteEntity(dest_tab, determineWriteType(ltd, dest));
if (!outputs.add(output)) {
- if(!((this instanceof MergeSemanticAnalyzer) &&
- conf.getBoolVar(ConfVars.MERGE_SPLIT_UPDATE))) {
+ if(!allowOutputMultipleTimes()) {
/**
* Merge stmt with early split update may create several (2) writes to the same
* table with the same {@link WriteType}, e.g. if original Merge stmt has both update and
@@ -8411,6 +8412,10 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
return output;
}
+ protected boolean allowOutputMultipleTimes() {
+ return false;
+ }
+
private void checkExternalTable(Table dest_tab) throws SemanticException {
if ((!conf.getBoolVar(HiveConf.ConfVars.HIVE_INSERT_INTO_EXTERNAL_TABLES)) &&
(dest_tab.getTableType().equals(TableType.EXTERNAL_TABLE))) {
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 3b215a7acfe..f83f36d71a6 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
@@ -17,6 +17,8 @@
*/
package org.apache.hadoop.hive.ql.parse;
+import java.util.ArrayList;
+import java.util.Collections;
import java.util.HashMap;
import java.util.LinkedHashSet;
import java.util.List;
@@ -24,6 +26,8 @@ import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
+
+import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hadoop.hive.metastore.api.FieldSchema;
import org.apache.hadoop.hive.ql.Context;
import org.apache.hadoop.hive.ql.ErrorMsg;
@@ -48,13 +52,22 @@ public class UpdateDeleteSemanticAnalyzer extends RewriteSemanticAnalyzer {
super(queryState);
}
- protected void analyze(ASTNode tree) throws SemanticException {
+ @Override
+ protected ASTNode getTargetTableNode(ASTNode tree) {
+ // The first child should be the table we are updating / deleting from
+ ASTNode tabName = (ASTNode)tree.getChild(0);
+ assert tabName.getToken().getType() == HiveParser.TOK_TABNAME :
+ "Expected tablename as first child of " + operation + " but found " + tabName.getName();
+ return tabName;
+ }
+
+ protected void analyze(ASTNode tree, Table table, ASTNode tabNameNode) throws SemanticException {
switch (tree.getToken().getType()) {
case HiveParser.TOK_DELETE_FROM:
- analyzeDelete(tree);
+ analyzeDelete(tree, table, tabNameNode);
break;
case HiveParser.TOK_UPDATE_TABLE:
- analyzeUpdate(tree);
+ analyzeUpdate(tree, table, tabNameNode);
break;
default:
throw new RuntimeException("Asked to parse token " + tree.getName() + " in " +
@@ -62,14 +75,20 @@ public class UpdateDeleteSemanticAnalyzer extends RewriteSemanticAnalyzer {
}
}
- private void analyzeUpdate(ASTNode tree) throws SemanticException {
+ private void analyzeUpdate(ASTNode tree, Table mTable, ASTNode tabNameNode) throws SemanticException {
operation = Context.Operation.UPDATE;
- reparseAndSuperAnalyze(tree);
+ boolean nonNativeAcid = AcidUtils.isNonNativeAcidTable(mTable);
+
+ if (HiveConf.getBoolVar(queryState.getConf(), HiveConf.ConfVars.SPLIT_UPDATE) && !nonNativeAcid) {
+ analyzeSplitUpdate(tree, mTable, tabNameNode);
+ } else {
+ reparseAndSuperAnalyze(tree, mTable, tabNameNode);
+ }
}
- private void analyzeDelete(ASTNode tree) throws SemanticException {
+ private void analyzeDelete(ASTNode tree, Table mTable, ASTNode tabNameNode) throws SemanticException {
operation = Context.Operation.DELETE;
- reparseAndSuperAnalyze(tree);
+ reparseAndSuperAnalyze(tree, mTable, tabNameNode);
}
/**
@@ -91,20 +110,15 @@ public class UpdateDeleteSemanticAnalyzer extends RewriteSemanticAnalyzer {
* The sort by clause is put in there so that records come out in the right order to enable
* merge on read.
*/
- private void reparseAndSuperAnalyze(ASTNode tree) throws SemanticException {
+ private void reparseAndSuperAnalyze(ASTNode tree, Table mTable, ASTNode tabNameNode) throws SemanticException {
List<? extends Node> children = tree.getChildren();
- // The first child should be the table we are updating / deleting from
- ASTNode tabName = (ASTNode)children.get(0);
- assert tabName.getToken().getType() == HiveParser.TOK_TABNAME :
- "Expected tablename as first child of " + operation + " but found " + tabName.getName();
- Table mTable = getTargetTable(tabName);
- validateTxnManager(mTable);
- validateTargetTable(mTable);
+ // save the operation type into the query state
+ SessionStateUtil.addResource(conf, Context.Operation.class.getSimpleName(), operation.name());
StringBuilder rewrittenQueryStr = new StringBuilder();
rewrittenQueryStr.append("insert into table ");
- rewrittenQueryStr.append(getFullTableNameForSQL(tabName));
+ rewrittenQueryStr.append(getFullTableNameForSQL(tabNameNode));
addPartitionColsToInsert(mTable.getPartCols(), rewrittenQueryStr);
boolean nonNativeAcid = AcidUtils.isNonNativeAcidTable(mTable);
@@ -149,9 +163,9 @@ public class UpdateDeleteSemanticAnalyzer extends RewriteSemanticAnalyzer {
}
}
- addPartitionColsToSelect(mTable.getPartCols(), rewrittenQueryStr, null);
+ addPartitionColsToSelect(mTable.getPartCols(), rewrittenQueryStr);
rewrittenQueryStr.append(" from ");
- rewrittenQueryStr.append(getFullTableNameForSQL(tabName));
+ rewrittenQueryStr.append(getFullTableNameForSQL(tabNameNode));
ASTNode where = null;
int whereIndex = deleting() ? 1 : 2;
@@ -219,34 +233,13 @@ public class UpdateDeleteSemanticAnalyzer extends RewriteSemanticAnalyzer {
}
}
- // Patch up the projection list for updates, putting back the original set expressions.
if (updating() && setColExprs != null) {
- // Walk through the projection list and replace the column names with the
- // expressions from the original update. Under the TOK_SELECT (see above) the structure
- // looks like:
- // TOK_SELECT -> TOK_SELEXPR -> expr
- // \-> TOK_SELEXPR -> expr ...
- ASTNode rewrittenSelect = (ASTNode)rewrittenInsert.getChildren().get(1);
- assert rewrittenSelect.getToken().getType() == HiveParser.TOK_SELECT :
- "Expected TOK_SELECT as second child of TOK_INSERT but found " +
- rewrittenSelect.getName();
- for (Map.Entry<Integer, ASTNode> entry : setColExprs.entrySet()) {
- ASTNode selExpr = (ASTNode)rewrittenSelect.getChildren().get(entry.getKey());
- assert selExpr.getToken().getType() == HiveParser.TOK_SELEXPR :
- "Expected child of TOK_SELECT to be TOK_SELEXPR but was " + selExpr.getName();
- // Now, change it's child
- selExpr.setChild(0, entry.getValue());
- }
+ patchProjectionForUpdate(rewrittenInsert, setColExprs);
}
- try {
- useSuper = true;
- // Note: this will overwrite this.ctx with rewrittenCtx
- rewrittenCtx.setEnableUnparse(false);
- super.analyze(rewrittenTree, rewrittenCtx);
- } finally {
- useSuper = false;
- }
+ // Note: this will overwrite this.ctx with rewrittenCtx
+ rewrittenCtx.setEnableUnparse(false);
+ analyzeRewrittenTree(rewrittenTree, rewrittenCtx);
updateOutputs(mTable);
@@ -264,9 +257,104 @@ public class UpdateDeleteSemanticAnalyzer extends RewriteSemanticAnalyzer {
}
}
- private void validateTxnManager(Table mTable) throws SemanticException {
- if (!AcidUtils.acidTableWithoutTransactions(mTable) && !getTxnMgr().supportsAcid()) {
- throw new SemanticException(ErrorMsg.ACID_OP_ON_NONACID_TXNMGR.getMsg());
+ private void analyzeSplitUpdate(ASTNode tree, Table mTable, ASTNode tabNameNode) throws SemanticException {
+ operation = Context.Operation.UPDATE;
+
+ List<? extends Node> children = tree.getChildren();
+
+ ASTNode where = null;
+ int whereIndex = 2;
+ if (children.size() > whereIndex) {
+ where = (ASTNode) children.get(whereIndex);
+ assert where.getToken().getType() == HiveParser.TOK_WHERE :
+ "Expected where clause, but found " + where.getName();
+ }
+
+ Set<String> setRCols = new LinkedHashSet<>();
+// TOK_UPDATE_TABLE
+// TOK_TABNAME
+// ...
+// TOK_SET_COLUMNS_CLAUSE <- The set list from update should be the second child (index 1)
+ assert children.size() >= 2 : "Expected update token to have at least two children";
+ ASTNode setClause = (ASTNode) children.get(1);
+ Map<String, ASTNode> setCols = collectSetColumnsAndExpressions(setClause, setRCols, mTable);
+ Map<Integer, ASTNode> setColExprs = new HashMap<>(setClause.getChildCount());
+
+ List<FieldSchema> nonPartCols = mTable.getCols();
+ Map<String, String> colNameToDefaultConstraint = getColNameToDefaultValueMap(mTable);
+ List<String> values = new ArrayList<>(mTable.getCols().size());
+ StringBuilder rewrittenQueryStr = createRewrittenQueryStrBuilder();
+ rewrittenQueryStr.append("(SELECT ROW__ID");
+ for (int i = 0; i < nonPartCols.size(); i++) {
+ rewrittenQueryStr.append(',');
+ String name = nonPartCols.get(i).getName();
+ ASTNode setCol = setCols.get(name);
+ String identifier = HiveUtils.unparseIdentifier(name, this.conf);
+
+ if (setCol != null) {
+ if (setCol.getType() == HiveParser.TOK_TABLE_OR_COL &&
+ setCol.getChildCount() == 1 && setCol.getChild(0).getType() == HiveParser.TOK_DEFAULT_VALUE) {
+ rewrittenQueryStr.append(colNameToDefaultConstraint.get(name));
+ } else {
+ rewrittenQueryStr.append(identifier);
+ // This is one of the columns we're setting, record it's position so we can come back
+ // later and patch it up. 0th is ROW_ID
+ setColExprs.put(i + 1, setCol);
+ }
+ } else {
+ rewrittenQueryStr.append(identifier);
+ }
+ rewrittenQueryStr.append(" AS ");
+ rewrittenQueryStr.append(identifier);
+
+ values.add("s." + identifier);
+ }
+ addPartitionColsToSelect(mTable.getPartCols(), rewrittenQueryStr);
+ addPartitionColsAsValues(mTable.getPartCols(), "s", values);
+ rewrittenQueryStr.append(" FROM ").append(getFullTableNameForSQL(tabNameNode)).append(") s\n");
+
+ appendInsertBranch(rewrittenQueryStr, null, values);
+ appendDeleteBranch(rewrittenQueryStr, null, "s", Collections.singletonList("s.ROW__ID "));
+
+ appendSortBy(rewrittenQueryStr, Collections.singletonList("s.ROW__ID "));
+
+ ReparseResult rr = parseRewrittenQuery(rewrittenQueryStr, ctx.getCmd());
+ Context rewrittenCtx = rr.rewrittenCtx;
+ ASTNode rewrittenTree = rr.rewrittenTree;
+
+ ASTNode rewrittenInsert = new ASTSearcher().simpleBreadthFirstSearch(
+ rewrittenTree, HiveParser.TOK_FROM, HiveParser.TOK_SUBQUERY, HiveParser.TOK_INSERT);
+
+ rewrittenCtx.setOperation(Context.Operation.UPDATE);
+ rewrittenCtx.addDestNamePrefix(1, Context.DestClausePrefix.INSERT);
+ rewrittenCtx.addDeleteOfUpdateDestNamePrefix(2, Context.DestClausePrefix.DELETE);
+
+ if (where != null) {
+ rewrittenInsert.addChild(where);
+ }
+
+ patchProjectionForUpdate(rewrittenInsert, setColExprs);
+
+ try {
+ useSuper = true;
+ // Note: this will overwrite this.ctx with rewrittenCtx
+ rewrittenCtx.setEnableUnparse(false);
+ super.analyze(rewrittenTree, rewrittenCtx);
+ } finally {
+ useSuper = false;
+ }
+
+ updateOutputs(mTable);
+
+ setUpAccessControlInfoForUpdate(mTable, setCols);
+
+ // Add the setRCols to the input list
+ if (columnAccessInfo == null) { //assuming this means we are not doing Auth
+ return;
+ }
+
+ for (String colName : setRCols) {
+ columnAccessInfo.add(Table.getCompleteName(mTable.getDbName(), mTable.getTableName()), colName);
}
}
@@ -276,4 +364,9 @@ public class UpdateDeleteSemanticAnalyzer extends RewriteSemanticAnalyzer {
private boolean deleting() {
return operation == Context.Operation.DELETE;
}
+
+ @Override
+ protected boolean allowOutputMultipleTimes() {
+ return conf.getBoolVar(HiveConf.ConfVars.SPLIT_UPDATE);
+ }
}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java
index f2742bdfe54..c801d0c9cbf 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java
@@ -125,6 +125,7 @@ public class FileSinkDesc extends AbstractOperatorDesc implements IStatsGatherDe
private boolean isDirectInsert = false;
private AcidUtils.Operation acidOperation = null;
+ private boolean deleteOfSplitUpdate;
private boolean isQuery = false;
@@ -140,7 +141,7 @@ public class FileSinkDesc extends AbstractOperatorDesc implements IStatsGatherDe
final boolean multiFileSpray, final boolean canBeMerged, final int numFiles, final int totalFiles,
final List<ExprNodeDesc> partitionCols, final DynamicPartitionCtx dpCtx, Path destPath, Long mmWriteId,
boolean isMmCtas, boolean isInsertOverwrite, boolean isQuery, boolean isCTASorCM, boolean isDirectInsert,
- AcidUtils.Operation acidOperation) {
+ AcidUtils.Operation acidOperation, boolean deleteOfSplitUpdate) {
this.dirName = dirName;
setTableInfo(tableInfo);
this.compressed = compressed;
@@ -160,6 +161,7 @@ public class FileSinkDesc extends AbstractOperatorDesc implements IStatsGatherDe
this.isCTASorCM = isCTASorCM;
this.isDirectInsert = isDirectInsert;
this.acidOperation = acidOperation;
+ this.deleteOfSplitUpdate = deleteOfSplitUpdate;
}
public FileSinkDesc(final Path dirName, final TableDesc tableInfo,
@@ -181,7 +183,7 @@ public class FileSinkDesc extends AbstractOperatorDesc implements IStatsGatherDe
public Object clone() throws CloneNotSupportedException {
FileSinkDesc ret = new FileSinkDesc(dirName, tableInfo, compressed, destTableId, multiFileSpray, canBeMerged,
numFiles, totalFiles, partitionCols, dpCtx, destPath, mmWriteId, isMmCtas, isInsertOverwrite, isQuery,
- isCTASorCM, isDirectInsert, acidOperation);
+ isCTASorCM, isDirectInsert, acidOperation, deleteOfSplitUpdate);
ret.setCompressCodec(compressCodec);
ret.setCompressType(compressType);
ret.setGatherStats(gatherStats);
@@ -251,11 +253,15 @@ public class FileSinkDesc extends AbstractOperatorDesc implements IStatsGatherDe
public void setAcidOperation(AcidUtils.Operation acidOperation) {
this.acidOperation = acidOperation;
- }
+ }
+
+ public AcidUtils.Operation getAcidOperation() {
+ return acidOperation;
+ }
- public AcidUtils.Operation getAcidOperation() {
- return acidOperation;
- }
+ public boolean isDeleteOfSplitUpdate() {
+ return deleteOfSplitUpdate;
+ }
@Explain(displayName = "directory", explainLevels = { Level.EXTENDED })
public Path getDirName() {
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java
index e5830be9ada..692e76f5b66 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java
@@ -337,7 +337,7 @@ public class TestTxnCommands2 extends TxnCommandsBaseForTests {
{"{\"writeid\":0,\"bucketid\":536936448,\"rowid\":2}\t0\t13", "bucket_00001"},
{"{\"writeid\":10000001,\"bucketid\":536936448,\"rowid\":0}\t0\t15", "bucket_00001"},
{"{\"writeid\":10000003,\"bucketid\":536936448,\"rowid\":0}\t0\t17", "bucket_00001"},
- {"{\"writeid\":10000002,\"bucketid\":536936448,\"rowid\":0}\t0\t120", "bucket_00001"},
+ {"{\"writeid\":10000002,\"bucketid\":536936449,\"rowid\":0}\t0\t120", "bucket_00001"},
{"{\"writeid\":0,\"bucketid\":536936448,\"rowid\":1}\t1\t2", "bucket_00001"},
{"{\"writeid\":0,\"bucketid\":536936448,\"rowid\":5}\t1\t4", "bucket_00001"},
{"{\"writeid\":0,\"bucketid\":536936448,\"rowid\":4}\t1\t5", "bucket_00001"},
@@ -732,7 +732,7 @@ public class TestTxnCommands2 extends TxnCommandsBaseForTests {
FileStatus[] buckets = fs.listStatus(status[i].getPath(), FileUtils.HIDDEN_FILES_PATH_FILTER);
Arrays.sort(buckets);
if (numDelta == 1) {
- Assert.assertEquals("delta_10000001_10000001_0000", status[i].getPath().getName());
+ Assert.assertEquals("delta_10000001_10000001_0001", status[i].getPath().getName());
Assert.assertEquals(BUCKET_COUNT - 1, buckets.length);
Assert.assertEquals("bucket_00001_0", buckets[0].getPath().getName());
} else if (numDelta == 2) {
@@ -923,7 +923,7 @@ public class TestTxnCommands2 extends TxnCommandsBaseForTests {
FileStatus[] buckets = fs.listStatus(parent, FileUtils.HIDDEN_FILES_PATH_FILTER);
Arrays.sort(buckets);
if (numDelta == 1) {
- Assert.assertEquals("delta_10000001_10000001_0000", parent.getName());
+ Assert.assertEquals("delta_10000001_10000001_0001", parent.getName());
Assert.assertEquals(BUCKET_COUNT - 1, buckets.length);
Assert.assertEquals("bucket_00001_0", buckets[0].getPath().getName());
} else if (numDelta == 2) {
@@ -2716,7 +2716,7 @@ public class TestTxnCommands2 extends TxnCommandsBaseForTests {
hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEFAILLOADDYNAMICPARTITION, true);
runStatementOnDriverWithAbort("update " + Table.ACIDTBLPART + " set b=a+2 where a<5");
hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEFAILLOADDYNAMICPARTITION, false);
- verifyDeltaDirAndResult(3, Table.ACIDTBLPART.toString(), "p=p1", resultData1);
+ verifyDeltaDirAndResult(2, Table.ACIDTBLPART.toString(), "p=p1", resultData1);
verifyDeleteDeltaDir(1, Table.ACIDTBLPART.toString(), "p=p1");
int count = TestTxnDbUtil
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands3.java b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands3.java
index f3a261b4ea7..9537f1c083c 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands3.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands3.java
@@ -85,8 +85,8 @@ public class TestTxnCommands3 extends TxnCommandsBaseForTests {
String[][] expected = new String[][] {
{"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":0}\t1\t2",
"s/delta_0000001_0000001_0000/bucket_00000_0"},
- {"{\"writeid\":2,\"bucketid\":536870912,\"rowid\":0}\t4\t6",
- "s/delta_0000002_0000002_0000/bucket_00000_0"}};
+ {"{\"writeid\":2,\"bucketid\":536870913,\"rowid\":0}\t4\t6",
+ "s/delta_0000002_0000002_0001/bucket_00000_0"}};
checkResult(expected, testQuery, false, "check data", LOG);
@@ -293,15 +293,15 @@ public class TestTxnCommands3 extends TxnCommandsBaseForTests {
runStatementOnDriver("update acid_uap set b = 'fred'");
String[][] expected2 = new String[][]{
- {"{\"writeid\":3,\"bucketid\":536936448,\"rowid\":0}\t1\tfred\ttoday",
- "warehouse/acid_uap/ds=today/delta_0000003_0000003_0000/bucket_00001_0"},
- {"{\"writeid\":3,\"bucketid\":536870912,\"rowid\":0}\t2\tfred\ttoday",
- "warehouse/acid_uap/ds=today/delta_0000003_0000003_0000/bucket_00000_0"},
-
- {"{\"writeid\":3,\"bucketid\":536936448,\"rowid\":0}\t1\tfred\ttomorrow",
- "warehouse/acid_uap/ds=tomorrow/delta_0000003_0000003_0000/bucket_00001_0"},
- {"{\"writeid\":3,\"bucketid\":536870912,\"rowid\":0}\t2\tfred\ttomorrow",
- "warehouse/acid_uap/ds=tomorrow/delta_0000003_0000003_0000/bucket_00000_0"}};
+ {"{\"writeid\":3,\"bucketid\":536936449,\"rowid\":0}\t1\tfred\ttoday",
+ "warehouse/acid_uap/ds=today/delta_0000003_0000003_0001/bucket_00001_0"},
+ {"{\"writeid\":3,\"bucketid\":536870913,\"rowid\":0}\t2\tfred\ttoday",
+ "warehouse/acid_uap/ds=today/delta_0000003_0000003_0001/bucket_00000_0"},
+
+ {"{\"writeid\":3,\"bucketid\":536936449,\"rowid\":0}\t1\tfred\ttomorrow",
+ "warehouse/acid_uap/ds=tomorrow/delta_0000003_0000003_0001/bucket_00001_0"},
+ {"{\"writeid\":3,\"bucketid\":536870913,\"rowid\":0}\t2\tfred\ttomorrow",
+ "warehouse/acid_uap/ds=tomorrow/delta_0000003_0000003_0001/bucket_00000_0"}};
checkResult(expected2, testQuery, isVectorized, "after update", LOG);
}
@Test
@@ -574,7 +574,7 @@ public class TestTxnCommands3 extends TxnCommandsBaseForTests {
"/t/delta_0000001_0000001_0000",
"/t/delta_0000002_0000002_0000",
"/t/delete_delta_0000003_0000003_0000",
- "/t/delta_0000003_0000003_0000",
+ "/t/delta_0000003_0000003_0001",
};
checkExpectedFiles(actualList, expectedList, warehousePath.toString());
//delete metadata about aborted txn from txn_components and files (if any)
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnConcatenate.java b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnConcatenate.java
index 1c16bb97608..598402f778e 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnConcatenate.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnConcatenate.java
@@ -54,10 +54,10 @@ public class TestTxnConcatenate extends TxnCommandsBaseForTests {
runStatementOnDriver("insert into " + Table.ACIDTBL + " values(5,6),(8,8)");
String testQuery = "select ROW__ID, a, b, INPUT__FILE__NAME from " + Table.ACIDTBL + " order by a, b";
String[][] expected = new String[][] {
- {"{\"writeid\":2,\"bucketid\":536936448,\"rowid\":0}\t1\t4",
- "acidtbl/delta_0000002_0000002_0000/bucket_00001_0"},
- {"{\"writeid\":2,\"bucketid\":536936448,\"rowid\":1}\t4\t4",
- "acidtbl/delta_0000002_0000002_0000/bucket_00001_0"},
+ {"{\"writeid\":2,\"bucketid\":536936449,\"rowid\":0}\t1\t4",
+ "acidtbl/delta_0000002_0000002_0001/bucket_00001_0"},
+ {"{\"writeid\":2,\"bucketid\":536936449,\"rowid\":1}\t4\t4",
+ "acidtbl/delta_0000002_0000002_0001/bucket_00001_0"},
{"{\"writeid\":3,\"bucketid\":536936448,\"rowid\":0}\t5\t6",
"acidtbl/delta_0000003_0000003_0000/bucket_00001_0"},
{"{\"writeid\":3,\"bucketid\":536936448,\"rowid\":1}\t8\t8",
@@ -78,9 +78,9 @@ public class TestTxnConcatenate extends TxnCommandsBaseForTests {
Assert.assertEquals(1, rsp.getCompactsSize());
Assert.assertEquals(TxnStore.CLEANING_RESPONSE, rsp.getCompacts().get(0).getState());
String[][] expected2 = new String[][] {
- {"{\"writeid\":2,\"bucketid\":536936448,\"rowid\":0}\t1\t4",
+ {"{\"writeid\":2,\"bucketid\":536936449,\"rowid\":0}\t1\t4",
"acidtbl/base_0000003_v0000021/bucket_00001"},
- {"{\"writeid\":2,\"bucketid\":536936448,\"rowid\":1}\t4\t4",
+ {"{\"writeid\":2,\"bucketid\":536936449,\"rowid\":1}\t4\t4",
"acidtbl/base_0000003_v0000021/bucket_00001"},
{"{\"writeid\":3,\"bucketid\":536936448,\"rowid\":0}\t5\t6",
"acidtbl/base_0000003_v0000021/bucket_00001"},
@@ -95,8 +95,8 @@ public class TestTxnConcatenate extends TxnCommandsBaseForTests {
runStatementOnDriver("insert into " + Table.ACIDTBLPART + " values(5,6,'p1'),(8,8,'p2')");
String testQuery = "select ROW__ID, a, b, INPUT__FILE__NAME from " + Table.ACIDTBLPART + " order by a, b";
String[][] expected = new String[][] {
- {"{\"writeid\":2,\"bucketid\":536936448,\"rowid\":0}\t1\t4",
- "acidtblpart/p=p1/delta_0000002_0000002_0000/bucket_00001_0"},
+ {"{\"writeid\":2,\"bucketid\":536936449,\"rowid\":0}\t1\t4",
+ "acidtblpart/p=p1/delta_0000002_0000002_0001/bucket_00001_0"},
{"{\"writeid\":1,\"bucketid\":536936448,\"rowid\":0}\t4\t5",
"acidtblpart/p=p2/delta_0000001_0000001_0000/bucket_00001_0"},
{"{\"writeid\":3,\"bucketid\":536936448,\"rowid\":0}\t5\t6",
@@ -119,7 +119,7 @@ public class TestTxnConcatenate extends TxnCommandsBaseForTests {
Assert.assertEquals(1, rsp.getCompactsSize());
Assert.assertEquals(TxnStore.CLEANING_RESPONSE, rsp.getCompacts().get(0).getState());
String[][] expected2 = new String[][] {
- {"{\"writeid\":2,\"bucketid\":536936448,\"rowid\":0}\t1\t4",
+ {"{\"writeid\":2,\"bucketid\":536936449,\"rowid\":0}\t1\t4",
"acidtblpart/p=p1/base_0000003_v0000021/bucket_00001"},
{"{\"writeid\":1,\"bucketid\":536936448,\"rowid\":0}\t4\t5",
"acidtblpart/p=p2/delta_0000001_0000001_0000/bucket_00001_0"},
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnExIm.java b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnExIm.java
index 34affbf0a77..95cf36f6e3b 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnExIm.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnExIm.java
@@ -338,8 +338,8 @@ target/tmp/org.apache.hadoop.hive.ql.TestTxnCommands-1521148657811/
"t/delta_0000001_0000001_0000/000000_0"},
{"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":1}\t3\t4",
"t/delta_0000001_0000001_0000/000000_0"},
- {"{\"writeid\":2,\"bucketid\":536870912,\"rowid\":0}\t0\t6",
- "t/delta_0000002_0000002_0000/bucket_00000_0"}};
+ {"{\"writeid\":2,\"bucketid\":536870913,\"rowid\":0}\t0\t6",
+ "t/delta_0000002_0000002_0001/bucket_00000_0"}};
checkResult(expected2, testQuery, isVectorized, "update imported table");
runStatementOnDriver("alter table T compact 'minor'");
@@ -349,7 +349,7 @@ target/tmp/org.apache.hadoop.hive.ql.TestTxnCommands-1521148657811/
".*t/delta_0000001_0000002_v000002[6-7]/bucket_00000"},
{"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":1}\t3\t4",
".*t/delta_0000001_0000002_v000002[6-7]/bucket_00000"},
- {"{\"writeid\":2,\"bucketid\":536870912,\"rowid\":0}\t0\t6",
+ {"{\"writeid\":2,\"bucketid\":536870913,\"rowid\":0}\t0\t6",
".*t/delta_0000001_0000002_v000002[6-7]/bucket_00000"}};
checkResult(expected3, testQuery, isVectorized, "minor compact imported table");
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnLoadData.java b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnLoadData.java
index caf2994cbef..a4d4aeff50e 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnLoadData.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnLoadData.java
@@ -118,20 +118,20 @@ public class TestTxnLoadData extends TxnCommandsBaseForTests {
runStatementOnDriver("update T set b = 17 where a = 1");
String[][] expected2 = new String[][]{
{"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":1}\t3\t4", "t/delta_0000001_0000001_0000/000000_0"},
- {"{\"writeid\":2,\"bucketid\":536870912,\"rowid\":0}\t1\t17", "t/delta_0000002_0000002_0000/bucket_00000_0"}
+ {"{\"writeid\":2,\"bucketid\":536870913,\"rowid\":0}\t1\t17", "t/delta_0000002_0000002_0001/bucket_00000_0"}
};
checkResult(expected2, testQuery, isVectorized, "update");
runStatementOnDriver("insert into T values(2,2)");
String[][] expectedInter2 = new String[][] {
{"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":1}\t3\t4", "t/delta_0000001_0000001_0000/000000_0"},
- {"{\"writeid\":2,\"bucketid\":536870912,\"rowid\":0}\t1\t17", "t/delta_0000002_0000002_0000/bucket_00000_0"},
+ {"{\"writeid\":2,\"bucketid\":536870913,\"rowid\":0}\t1\t17", "t/delta_0000002_0000002_0001/bucket_00000_0"},
{"{\"writeid\":3,\"bucketid\":536870912,\"rowid\":0}\t2\t2", "t/delta_0000003_0000003_0000/bucket_00000_0"}
};
checkResult(expectedInter2, testQuery, isVectorized, "insert");
runStatementOnDriver("delete from T where a = 3");
String[][] expectedInter3 = new String[][] {
- {"{\"writeid\":2,\"bucketid\":536870912,\"rowid\":0}\t1\t17", "t/delta_0000002_0000002_0000/bucket_00000_0"},
+ {"{\"writeid\":2,\"bucketid\":536870913,\"rowid\":0}\t1\t17", "t/delta_0000002_0000002_0001/bucket_00000_0"},
{"{\"writeid\":3,\"bucketid\":536870912,\"rowid\":0}\t2\t2", "t/delta_0000003_0000003_0000/bucket_00000_0"}
};
checkResult(expectedInter3, testQuery, isVectorized, "delete");
@@ -139,7 +139,7 @@ public class TestTxnLoadData extends TxnCommandsBaseForTests {
runStatementOnDriver("alter table T compact 'minor'");
TestTxnCommands2.runWorker(hiveConf);
String[][] expected3 = new String[][] {
- {"{\"writeid\":2,\"bucketid\":536870912,\"rowid\":0}\t1\t17", "t/delta_0000001_0000004_v0000034/bucket_00000"},
+ {"{\"writeid\":2,\"bucketid\":536870913,\"rowid\":0}\t1\t17", "t/delta_0000001_0000004_v0000034/bucket_00000"},
{"{\"writeid\":3,\"bucketid\":536870912,\"rowid\":0}\t2\t2", "t/delta_0000001_0000004_v0000034/bucket_00000"}
};
checkResult(expected3, testQuery, isVectorized, "delete compact minor");
@@ -163,8 +163,8 @@ public class TestTxnLoadData extends TxnCommandsBaseForTests {
runStatementOnDriver("delete from T where a = 3");//matches 2 rows
runStatementOnDriver("insert into T values(2,2)");
String[][] expected5 = new String[][]{
- {"{\"writeid\":7,\"bucketid\":536870912,\"rowid\":0}\t1\t17", "t/delta_0000007_0000007_0000/bucket_00000_0"},
- {"{\"writeid\":7,\"bucketid\":536870912,\"rowid\":1}\t1\t17", "t/delta_0000007_0000007_0000/bucket_00000_0"},
+ {"{\"writeid\":7,\"bucketid\":536870913,\"rowid\":0}\t1\t17", "t/delta_0000007_0000007_0001/bucket_00000_0"},
+ {"{\"writeid\":7,\"bucketid\":536936449,\"rowid\":0}\t1\t17", "t/delta_0000007_0000007_0001/bucket_00001_0"},
{"{\"writeid\":9,\"bucketid\":536870912,\"rowid\":0}\t2\t2", "t/delta_0000009_0000009_0000/bucket_00000_0"}
};
checkResult(expected5, testQuery, isVectorized, "load data inpath overwrite update");
@@ -173,8 +173,8 @@ public class TestTxnLoadData extends TxnCommandsBaseForTests {
runStatementOnDriver("alter table T compact 'major'");
TestTxnCommands2.runWorker(hiveConf);
String[][] expected6 = new String[][]{
- {"{\"writeid\":7,\"bucketid\":536870912,\"rowid\":0}\t1\t17", "t/base_0000009_v0000048/bucket_00000"},
- {"{\"writeid\":7,\"bucketid\":536870912,\"rowid\":1}\t1\t17", "t/base_0000009_v0000048/bucket_00000"},
+ {"{\"writeid\":7,\"bucketid\":536870913,\"rowid\":0}\t1\t17", "t/base_0000009_v0000048/bucket_00000"},
+ {"{\"writeid\":7,\"bucketid\":536936449,\"rowid\":0}\t1\t17", "t/base_0000009_v0000048/bucket_00001"},
{"{\"writeid\":9,\"bucketid\":536870912,\"rowid\":0}\t2\t2", "t/base_0000009_v0000048/bucket_00000"}
};
checkResult(expected6, testQuery, isVectorized, "load data inpath compact major");
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnNoBuckets.java b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnNoBuckets.java
index f0194947df4..26c08358f85 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnNoBuckets.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnNoBuckets.java
@@ -125,11 +125,11 @@ public class TestTxnNoBuckets extends TxnCommandsBaseForTests {
Assert.assertTrue(rs.get(1), rs.get(1).startsWith("{\"writeid\":1,\"bucketid\":536936448,\"rowid\":1}\t2\t2\t2\t"));
Assert.assertTrue(rs.get(1), rs.get(1).endsWith(NO_BUCKETS_TBL_NAME + "/delta_0000001_0000001_0000/bucket_00001_0"));
//so update has 1 writer, but which creates buckets where the new rows land
- Assert.assertTrue(rs.get(2), rs.get(2).startsWith("{\"writeid\":2,\"bucketid\":536870912,\"rowid\":0}\t0\t0\t17\t"));
- Assert.assertTrue(rs.get(2), rs.get(2).endsWith(NO_BUCKETS_TBL_NAME + "/delta_0000002_0000002_0000/bucket_00000_0"));
+ Assert.assertTrue(rs.get(2), rs.get(2).startsWith("{\"writeid\":2,\"bucketid\":536870913,\"rowid\":0}\t1\t1\t17\t"));
+ Assert.assertTrue(rs.get(2), rs.get(2).endsWith(NO_BUCKETS_TBL_NAME + "/delta_0000002_0000002_0001/bucket_00000_0"));
// update for "{\"writeid\":1,\"bucketid\":536936448,\"rowid\":0}\t1\t1\t1\t"
- Assert.assertTrue(rs.get(3), rs.get(3).startsWith("{\"writeid\":2,\"bucketid\":536936448,\"rowid\":0}\t1\t1\t17\t"));
- Assert.assertTrue(rs.get(3), rs.get(3).endsWith(NO_BUCKETS_TBL_NAME + "/delta_0000002_0000002_0000/bucket_00001_0"));
+ Assert.assertTrue(rs.get(3), rs.get(3).startsWith("{\"writeid\":2,\"bucketid\":536936449,\"rowid\":0}\t0\t0\t17\t"));
+ Assert.assertTrue(rs.get(3), rs.get(3).endsWith(NO_BUCKETS_TBL_NAME + "/delta_0000002_0000002_0001/bucket_00001_0"));
Set<String> expectedFiles = new HashSet<>();
//both delete events land in corresponding buckets to the original row-ids
@@ -137,8 +137,8 @@ public class TestTxnNoBuckets extends TxnCommandsBaseForTests {
expectedFiles.add(NO_BUCKETS_TBL_NAME + "/delete_delta_0000002_0000002_0000/bucket_00001_0");
expectedFiles.add(NO_BUCKETS_TBL_NAME + "/delta_0000001_0000001_0000/bucket_00000_0");
expectedFiles.add(NO_BUCKETS_TBL_NAME + "/delta_0000001_0000001_0000/bucket_00001_0");
- expectedFiles.add(NO_BUCKETS_TBL_NAME + "/delta_0000002_0000002_0000/bucket_00000_0");
- expectedFiles.add(NO_BUCKETS_TBL_NAME + "/delta_0000002_0000002_0000/bucket_00001_0");
+ expectedFiles.add(NO_BUCKETS_TBL_NAME + "/delta_0000002_0000002_0001/bucket_00000_0");
+ expectedFiles.add(NO_BUCKETS_TBL_NAME + "/delta_0000002_0000002_0001/bucket_00001_0");
//check that we get the right files on disk
assertExpectedFileSet(expectedFiles, getWarehouseDir() + "/" + NO_BUCKETS_TBL_NAME, NO_BUCKETS_TBL_NAME);
//todo: it would be nice to check the contents of the files... could use orc.FileDump - it has
@@ -168,8 +168,8 @@ public class TestTxnNoBuckets extends TxnCommandsBaseForTests {
*/
String expected[][] = {
- {"{\"writeid\":2,\"bucketid\":536870912,\"rowid\":0}\t0\t0\t17", NO_BUCKETS_TBL_NAME + "/base_0000002_v0000026/bucket_00000"},
- {"{\"writeid\":2,\"bucketid\":536936448,\"rowid\":0}\t1\t1\t17", NO_BUCKETS_TBL_NAME + "/base_0000002_v0000026/bucket_00001"},
+ {"{\"writeid\":2,\"bucketid\":536936449,\"rowid\":0}\t0\t0\t17", NO_BUCKETS_TBL_NAME + "/base_0000002_v0000026/bucket_00001"},
+ {"{\"writeid\":2,\"bucketid\":536870913,\"rowid\":0}\t1\t1\t17", NO_BUCKETS_TBL_NAME + "/base_0000002_v0000026/bucket_00000"},
{"{\"writeid\":1,\"bucketid\":536936448,\"rowid\":1}\t2\t2\t2", NO_BUCKETS_TBL_NAME + "/base_0000002_v0000026/bucket_00001"},
{"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":1}\t3\t3\t3", NO_BUCKETS_TBL_NAME + "/base_0000002_v0000026/bucket_00000"}
};
@@ -184,8 +184,8 @@ public class TestTxnNoBuckets extends TxnCommandsBaseForTests {
expectedFiles.add(NO_BUCKETS_TBL_NAME + "/delete_delta_0000002_0000002_0000/bucket_00001_0");
expectedFiles.add(NO_BUCKETS_TBL_NAME + "/delta_0000001_0000001_0000/bucket_00000_0");
expectedFiles.add(NO_BUCKETS_TBL_NAME + "/delta_0000001_0000001_0000/bucket_00001_0");
- expectedFiles.add(NO_BUCKETS_TBL_NAME + "/delta_0000002_0000002_0000/bucket_00000_0");
- expectedFiles.add(NO_BUCKETS_TBL_NAME + "/delta_0000002_0000002_0000/bucket_00001_0");
+ expectedFiles.add(NO_BUCKETS_TBL_NAME + "/delta_0000002_0000002_0001/bucket_00000_0");
+ expectedFiles.add(NO_BUCKETS_TBL_NAME + "/delta_0000002_0000002_0001/bucket_00001_0");
expectedFiles.add(NO_BUCKETS_TBL_NAME + "/base_0000002_v0000026/bucket_00000");
expectedFiles.add(NO_BUCKETS_TBL_NAME + "/base_0000002_v0000026/bucket_00001");
assertExpectedFileSet(expectedFiles, getWarehouseDir() + "/" + NO_BUCKETS_TBL_NAME, NO_BUCKETS_TBL_NAME);
@@ -465,7 +465,7 @@ ekoifman:apache-hive-3.0.0-SNAPSHOT-bin ekoifman$ tree /Users/ekoifman/dev/hiver
{"{\"writeid\":0,\"bucketid\":536870912,\"rowid\":4}\t20\t40", "warehouse/t/HIVE_UNION_SUBDIR_15/000000_0"},
{"{\"writeid\":0,\"bucketid\":536870912,\"rowid\":5}\t50\t60", "warehouse/t/HIVE_UNION_SUBDIR_16/000000_0"},
// update for "{\"writeid\":0,\"bucketid\":536936448,\"rowid\":1}\t60\t80"
- {"{\"writeid\":10000001,\"bucketid\":536936448,\"rowid\":0}\t60\t88", "warehouse/t/delta_10000001_10000001_0000/bucket_00001_0"},
+ {"{\"writeid\":10000001,\"bucketid\":536870913,\"rowid\":0}\t60\t88", "warehouse/t/delta_10000001_10000001_0001/bucket_00000_0"},
};
rs = runStatementOnDriver("select ROW__ID, a, b, INPUT__FILE__NAME from T order by a, b, INPUT__FILE__NAME");
checkExpected(rs, expected3,"after converting to acid (no compaction with updates)");
@@ -493,8 +493,8 @@ ekoifman:apache-hive-3.0.0-SNAPSHOT-bin ekoifman$ tree /Users/ekoifman/dev/hiver
"warehouse/t/base_10000002_v0000030/bucket_00000"},
{"{\"writeid\":0,\"bucketid\":536870912,\"rowid\":5}\t50\t60",
"warehouse/t/base_10000002_v0000030/bucket_00000"},
- {"{\"writeid\":10000001,\"bucketid\":536936448,\"rowid\":0}\t60\t88",
- "warehouse/t/base_10000002_v0000030/bucket_00001"},
+ {"{\"writeid\":10000001,\"bucketid\":536870913,\"rowid\":0}\t60\t88",
+ "warehouse/t/base_10000002_v0000030/bucket_00000"},
};
checkExpected(rs, expected4,"after major compact");
}
@@ -572,8 +572,8 @@ ekoifman:apache-hive-3.0.0-SNAPSHOT-bin ekoifman$ tree /Users/ekoifman/dev/hiver
"bucket_00000", "bucket_00000_0"},
{"{\"writeid\":10000003,\"bucketid\":536870912,\"rowid\":0}\t0\t17",
"bucket_00000", "bucket_00000_0"},
- {"{\"writeid\":10000002,\"bucketid\":536870912,\"rowid\":0}\t0\t120",
- "bucket_00000", "bucket_00000_0"},
+ {"{\"writeid\":10000002,\"bucketid\":536936449,\"rowid\":0}\t0\t120",
+ "bucket_00001", "bucket_00001_0"},
{"{\"writeid\":0,\"bucketid\":536870912,\"rowid\":0}\t1\t2",
"bucket_00000", "000000_0"},
{"{\"writeid\":0,\"bucketid\":536870912,\"rowid\":4}\t1\t4",
@@ -728,7 +728,7 @@ ekoifman:apache-hive-3.0.0-SNAPSHOT-bin ekoifman$ tree /Users/ekoifman/dev/hiver
query = "select ROW__ID, b from T where b > 0 order by a";
rs = runStatementOnDriver(query);
String[][] expected4 = {
- {"{\"writeid\":10000001,\"bucketid\":536870912,\"rowid\":0}","17"},
+ {"{\"writeid\":10000001,\"bucketid\":536870913,\"rowid\":0}","17"},
{"{\"writeid\":0,\"bucketid\":536870912,\"rowid\":1}","4"},
{"{\"writeid\":0,\"bucketid\":536870912,\"rowid\":2}","6"},
{"{\"writeid\":0,\"bucketid\":536870912,\"rowid\":3}","8"},
@@ -749,7 +749,7 @@ ekoifman:apache-hive-3.0.0-SNAPSHOT-bin ekoifman$ tree /Users/ekoifman/dev/hiver
query = "select ROW__ID, a, b, INPUT__FILE__NAME from T where b > 0 order by a, b";
rs = runStatementOnDriver(query);
String[][] expected5 = {//the row__ids are the same after compaction
- {"{\"writeid\":10000001,\"bucketid\":536870912,\"rowid\":0}\t1\t17",
+ {"{\"writeid\":10000001,\"bucketid\":536870913,\"rowid\":0}\t1\t17",
"warehouse/t/base_10000001_v0000030/bucket_00000"},
{"{\"writeid\":0,\"bucketid\":536870912,\"rowid\":1}\t2\t4",
"warehouse/t/base_10000001_v0000030/bucket_00000"},
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TxnCommandsBaseForTests.java b/ql/src/test/org/apache/hadoop/hive/ql/TxnCommandsBaseForTests.java
index 88b5e0014eb..4c0190d5f94 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/TxnCommandsBaseForTests.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/TxnCommandsBaseForTests.java
@@ -124,7 +124,7 @@ public abstract class TxnCommandsBaseForTests {
.setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER,
"org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory");
hiveConf.setBoolVar(HiveConf.ConfVars.MERGE_CARDINALITY_VIOLATION_CHECK, true);
- HiveConf.setBoolVar(hiveConf, HiveConf.ConfVars.MERGE_SPLIT_UPDATE, true);
+ HiveConf.setBoolVar(hiveConf, HiveConf.ConfVars.SPLIT_UPDATE, true);
hiveConf.setBoolVar(HiveConf.ConfVars.HIVESTATSCOLAUTOGATHER, false);
hiveConf.setBoolean("mapred.input.dir.recursive", true);
TestTxnDbUtil.setConfValues(hiveConf);
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestFileSinkOperator.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestFileSinkOperator.java
index 40b370407b8..05d8f564860 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestFileSinkOperator.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestFileSinkOperator.java
@@ -286,7 +286,7 @@ public class TestFileSinkOperator {
DynamicPartitionCtx dpCtx = new DynamicPartitionCtx(partColMap, "Sunday", 100);
//todo: does this need the finalDestination?
desc = new FileSinkDesc(basePath, tableDesc, false, 1, false,
- false, 1, 1, partCols, dpCtx, null, null, false, false, false, false, false, writeType);
+ false, 1, 1, partCols, dpCtx, null, null, false, false, false, false, false, writeType, false);
} else {
desc = new FileSinkDesc(basePath, tableDesc, false);
}
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java b/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java
index a28151ecacd..82ffc315fbe 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java
@@ -1104,7 +1104,7 @@ public class TestDbTxnManager2 extends DbTxnManagerEndToEndTestBase{
long txnId = txnMgr.getCurrentTxnId();
txnMgr.acquireLocks(driver.getPlan(), ctx, "Known");
List<ShowLocksResponseElement> locks = getLocks(txnMgr);
- Assert.assertEquals("Unexpected lock count", 1, locks.size());
+ Assert.assertEquals("Unexpected lock count", 2, locks.size());
checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB_PART", "p=blah", locks);
HiveTxnManager txnMgr2 = TxnManagerFactory.getTxnManagerFactory().getTxnManager(conf);
swapTxnManager(txnMgr2);
@@ -1112,7 +1112,7 @@ public class TestDbTxnManager2 extends DbTxnManagerEndToEndTestBase{
long txnId2 = txnMgr2.getCurrentTxnId();
((DbTxnManager)txnMgr2).acquireLocks(driver.getPlan(), ctx, "Unknown", false);
locks = getLocks(txnMgr2); //should not matter which txnMgr is used here
- Assert.assertEquals("Unexpected lock count", 2, locks.size());
+ Assert.assertEquals("Unexpected lock count", 4, locks.size());
checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB_PART", "p=blah", locks);
checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB_PART", "p=blah", locks);
long writeId = txnMgr.getTableWriteId("default", "TAB_PART");
@@ -1261,12 +1261,12 @@ public class TestDbTxnManager2 extends DbTxnManagerEndToEndTestBase{
driver.compileAndRespond("update TAB_PART set b = 7 where p = 'blah'", true);
txnMgr.acquireLocks(driver.getPlan(), ctx, "Known");
List<ShowLocksResponseElement> locks = getLocks(txnMgr);
- Assert.assertEquals("Unexpected lock count", 1, locks.size());
+ Assert.assertEquals("Unexpected lock count", 2, locks.size());
checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB_PART", "p=blah", locks);
driver.compileAndRespond("update TAB_PART set b = 7 where p = 'blah'", true);
((DbTxnManager)txnMgr2).acquireLocks(driver.getPlan(), ctx, "Unknown", false);
locks = getLocks(txnMgr2); //should not matter which txnMgr is used here
- Assert.assertEquals("Unexpected lock count", 2, locks.size());
+ Assert.assertEquals("Unexpected lock count", 4, locks.size());
checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB_PART", "p=blah", locks);
checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB_PART", "p=blah", locks);
txnMgr.rollbackTxn();
@@ -1343,7 +1343,7 @@ public class TestDbTxnManager2 extends DbTxnManagerEndToEndTestBase{
driver.compileAndRespond("update tab2 set b = 7 where p='two'", true);
txnMgr2.acquireLocks(driver.getPlan(), ctx, "T2");
List<ShowLocksResponseElement> locks = getLocks(txnMgr2);
- Assert.assertEquals("Unexpected lock count", 1, locks.size());
+ Assert.assertEquals("Unexpected lock count", 2, locks.size());
checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB2", "p=two", locks);
//now start concurrent txn
@@ -1351,7 +1351,7 @@ public class TestDbTxnManager2 extends DbTxnManagerEndToEndTestBase{
driver.compileAndRespond("update tab2 set b = 7 where p='one'", true);
((DbTxnManager)txnMgr).acquireLocks(driver.getPlan(), ctx, "T3", false);
locks = getLocks(txnMgr);
- Assert.assertEquals("Unexpected lock count", 2, locks.size());
+ Assert.assertEquals("Unexpected lock count", 4, locks.size());
checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB2", "p=two", locks);
checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB2", "p=one", locks);
//this simulates the completion of txnid:2
@@ -1363,7 +1363,7 @@ public class TestDbTxnManager2 extends DbTxnManagerEndToEndTestBase{
txnHandler.addDynamicPartitions(adp);
txnMgr2.commitTxn(); //txnid:idTxnUpdate1
locks = getLocks(txnMgr2);
- Assert.assertEquals("Unexpected lock count", 1, locks.size());
+ Assert.assertEquals("Unexpected lock count", 2, locks.size());
checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB2", "p=one", locks);
//completion of txnid:idTxnUpdate2
writeId = txnMgr.getTableWriteId("default", "tab2");
@@ -1398,7 +1398,7 @@ public class TestDbTxnManager2 extends DbTxnManagerEndToEndTestBase{
driver.compileAndRespond("update tab1 set b = 7 where b=1", true);
txnMgr2.acquireLocks(driver.getPlan(), ctx, "T5");
locks = getLocks(txnMgr2);
- Assert.assertEquals("Unexpected lock count", 2, locks.size());
+ Assert.assertEquals("Unexpected lock count", 3, locks.size());
checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=two", locks);
checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=one", locks);
@@ -1407,7 +1407,7 @@ public class TestDbTxnManager2 extends DbTxnManagerEndToEndTestBase{
driver.compileAndRespond("update tab1 set b = 7 where b = 2", true);
((DbTxnManager)txnMgr).acquireLocks(driver.getPlan(), ctx, "T6", false);
locks = getLocks(txnMgr);
- Assert.assertEquals("Unexpected lock count", 4, locks.size());
+ Assert.assertEquals("Unexpected lock count", 6, locks.size());
checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=two", locks);
checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=one", locks);
checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=two", locks);
@@ -1422,7 +1422,7 @@ public class TestDbTxnManager2 extends DbTxnManagerEndToEndTestBase{
txnMgr2.commitTxn(); //txnid:idTxnUpdate3
locks = getLocks(txnMgr);
- Assert.assertEquals("Unexpected lock count", 2, locks.size());
+ Assert.assertEquals("Unexpected lock count", 3, locks.size());
checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=two", locks);
checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=one", locks);
//completion of txnid:idTxnUpdate4
@@ -1462,7 +1462,7 @@ public class TestDbTxnManager2 extends DbTxnManagerEndToEndTestBase{
driver.compileAndRespond("update tab1 set b = 7 where b=1", true);
txnMgr2.acquireLocks(driver.getPlan(), ctx, "T2");
List<ShowLocksResponseElement> locks = getLocks(txnMgr2);
- Assert.assertEquals("Unexpected lock count", 2, locks.size());
+ Assert.assertEquals("Unexpected lock count", 3, locks.size());
checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=two", locks);
checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=one", locks);
@@ -1471,7 +1471,7 @@ public class TestDbTxnManager2 extends DbTxnManagerEndToEndTestBase{
driver.compileAndRespond("update tab1 set b = 7 where p='two'", true);
((DbTxnManager)txnMgr).acquireLocks(driver.getPlan(), ctx, "T3", false);
locks = getLocks(txnMgr);
- Assert.assertEquals("Unexpected lock count", 3, locks.size());
+ Assert.assertEquals("Unexpected lock count", 5, locks.size());
checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=two", locks);
checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=one", locks);
checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=two", locks);
@@ -1485,7 +1485,7 @@ public class TestDbTxnManager2 extends DbTxnManagerEndToEndTestBase{
txnMgr2.commitTxn(); //txnid:idTxnUpdate1
locks = getLocks(txnMgr);
- Assert.assertEquals("Unexpected lock count", 1, locks.size());
+ Assert.assertEquals("Unexpected lock count", 2, locks.size());
checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=two", locks);
//completion of txnid:idTxnUpdate2
writeId = txnMgr.getTableWriteId("default", "tab1");
@@ -1524,7 +1524,7 @@ public class TestDbTxnManager2 extends DbTxnManagerEndToEndTestBase{
long idTxnUpdate1 = txnMgr2.getCurrentTxnId();
txnMgr2.acquireLocks(driver.getPlan(), ctx, "T2");
List<ShowLocksResponseElement> locks = getLocks(txnMgr2);
- Assert.assertEquals("Unexpected lock count", 2, locks.size());
+ Assert.assertEquals("Unexpected lock count", 3, locks.size());
checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=two", locks);
checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=one", locks);
@@ -1534,7 +1534,7 @@ public class TestDbTxnManager2 extends DbTxnManagerEndToEndTestBase{
long idTxnDelete1 = txnMgr.getCurrentTxnId();
((DbTxnManager)txnMgr).acquireLocks(driver.getPlan(), ctx, "T3", false);
locks = getLocks(txnMgr);
- Assert.assertEquals("Unexpected lock count", 3, locks.size());
+ Assert.assertEquals("Unexpected lock count", 4, locks.size());
checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=two", locks);
checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=one", locks);
checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=two", locks);
@@ -1601,7 +1601,7 @@ public class TestDbTxnManager2 extends DbTxnManagerEndToEndTestBase{
driver.compileAndRespond("update tab1 set b = 7 where b=2", true);
txnMgr2.acquireLocks(driver.getPlan(), ctx, "T2");
List<ShowLocksResponseElement> locks = getLocks(txnMgr2);
- Assert.assertEquals("Unexpected lock count", 2, locks.size());
+ Assert.assertEquals("Unexpected lock count", 3, locks.size());
checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=two", locks);
checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=one", locks);
@@ -1610,7 +1610,7 @@ public class TestDbTxnManager2 extends DbTxnManagerEndToEndTestBase{
driver.compileAndRespond("delete from tab1 where p='two' and b=2", true);
((DbTxnManager)txnMgr).acquireLocks(driver.getPlan(), ctx, "T3", false);
locks = getLocks(txnMgr);
- Assert.assertEquals("Unexpected lock count", 3, locks.size());
+ Assert.assertEquals("Unexpected lock count", 4, locks.size());
checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=two", locks);
checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=one", locks);
checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=two", locks);
@@ -2183,7 +2183,7 @@ public class TestDbTxnManager2 extends DbTxnManagerEndToEndTestBase{
TestTxnDbUtil.queryToString(conf, "select * from \"WRITE_SET\""),
causeConflict ? 1 : 0, //Inserts are not tracked by WRITE_SET
TestTxnDbUtil.countQueryAgent(conf, "select count(*) from \"WRITE_SET\" where \"WS_TXNID\"=" + txnid1 +
- " and \"WS_OPERATION_TYPE\"=" + (causeConflict ? "'u'" : "'i'")));
+ " and \"WS_OPERATION_TYPE\"=" + (causeConflict ? "'d'" : "'i'")));
//re-check locks which were in Waiting state - should now be Acquired
((DbLockManager)txnMgr2.getLockManager()).checkLock(extLockId);
@@ -2217,7 +2217,7 @@ public class TestDbTxnManager2 extends DbTxnManagerEndToEndTestBase{
Assert.assertNotNull("Didn't get exception", expectedException);
Assert.assertEquals("Got wrong message code", ErrorMsg.TXN_ABORTED, expectedException.getCanonicalErrorMsg());
Assert.assertEquals("Exception msg didn't match",
- "Aborting [txnid:7,7] due to a write conflict on default/target committed by [txnid:6,7] d/u",
+ "Aborting [txnid:7,7] due to a write conflict on default/target committed by [txnid:6,7] d/d",
expectedException.getCause().getMessage());
} else {
Assert.assertEquals("WRITE_SET mismatch(" + JavaUtils.txnIdToString(txnid1) + "): " +
@@ -2765,7 +2765,7 @@ public class TestDbTxnManager2 extends DbTxnManagerEndToEndTestBase{
long txnId1 = txnMgr.getCurrentTxnId();
txnMgr.acquireLocks(driver.getPlan(), ctx, "T1");
List<ShowLocksResponseElement> locks = getLocks();
- Assert.assertEquals("Unexpected lock count", 2, locks.size());
+ Assert.assertEquals("Unexpected lock count", 3, locks.size());
checkLock((sharedWrite ? LockType.SHARED_WRITE : LockType.EXCL_WRITE),
LockState.ACQUIRED, "default", "target", "p=1/q=2", locks);
@@ -2782,7 +2782,7 @@ public class TestDbTxnManager2 extends DbTxnManagerEndToEndTestBase{
long txnid2 = txnMgr2.getCurrentTxnId();
txnMgr2.acquireLocks(driver.getPlan(), ctx, "T2", false);
locks = getLocks(txnMgr);
- Assert.assertEquals("Unexpected lock count", 7, locks.size());
+ Assert.assertEquals("Unexpected lock count", 8, locks.size());
/*
* W locks from T1 are still there, so all locks from T2 block.
* The Update part of Merge requests W locks for each existing partition in target.
@@ -2792,7 +2792,7 @@ public class TestDbTxnManager2 extends DbTxnManagerEndToEndTestBase{
"default", "source", null, locks);
long extLockId = checkLock((sharedWrite ? LockType.SHARED_WRITE : LockType.SHARED_READ),
(sharedWrite ? LockState.ACQUIRED : LockState.WAITING),
- "default", "target", null, locks).getLockid();
+ "default", "target", null, locks, sharedWrite).getLockid();
checkLock((sharedWrite ? LockType.SHARED_WRITE : LockType.EXCL_WRITE),
(sharedWrite ? LockState.ACQUIRED : LockState.WAITING),
diff --git a/ql/src/test/queries/clientpositive/bucket_num_reducers_acid.q b/ql/src/test/queries/clientpositive/bucket_num_reducers_acid.q
index a44668eb494..901464f3000 100644
--- a/ql/src/test/queries/clientpositive/bucket_num_reducers_acid.q
+++ b/ql/src/test/queries/clientpositive/bucket_num_reducers_acid.q
@@ -17,14 +17,11 @@ insert into bucket_nr_acid values(1,1);
-- txn X + 1 write to bucket0 + b1
insert into bucket_nr_acid values(0,0),(3,3);
+-- Expect 1-1 reducers for both insert and delete branches
+set VerifyNumReducersHook.num.reducers=2;
+
update bucket_nr_acid set b = -1;
set hive.exec.post.hooks=;
select * from bucket_nr_acid order by a, b;
drop table bucket_nr_acid;
-
-
-
-
-
-
diff --git a/ql/src/test/queries/clientpositive/split_update.q b/ql/src/test/queries/clientpositive/split_update.q
new file mode 100644
index 00000000000..799655e5b32
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/split_update.q
@@ -0,0 +1,14 @@
+set hive.support.concurrency=true;
+set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+
+
+create table acid_uami_n0(i int,
+ de decimal(5,2) constraint nn1 not null enforced,
+ vc varchar(128) constraint ch2 CHECK (de >= cast(i as decimal(5,2))) enforced)
+ clustered by (i) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true');
+
+set hive.split.update=true;
+explain update acid_uami_n0 set de = 893.14 where de = 103.00 or de = 119.00;
+
+set hive.split.update=false;
+explain update acid_uami_n0 set de = 893.14 where de = 103.00 or de = 119.00;
diff --git a/ql/src/test/results/clientnegative/authorization_update_noupdatepriv.q.out b/ql/src/test/results/clientnegative/authorization_update_noupdatepriv.q.out
index ad6d55122b5..751f59c356f 100644
--- a/ql/src/test/results/clientnegative/authorization_update_noupdatepriv.q.out
+++ b/ql/src/test/results/clientnegative/authorization_update_noupdatepriv.q.out
@@ -6,4 +6,4 @@ POSTHOOK: query: create table auth_noupd(i int, j int) clustered by (j) into 2 b
POSTHOOK: type: CREATETABLE
POSTHOOK: Output: database:default
POSTHOOK: Output: default@auth_noupd
-FAILED: HiveAccessControlException Permission denied: Principal [name=user1, type=USER] does not have following privileges for operation QUERY [[SELECT] on Object [type=TABLE_OR_VIEW, name=default.auth_noupd], [UPDATE] on Object [type=TABLE_OR_VIEW, name=default.auth_noupd]]
+FAILED: HiveAccessControlException Permission denied: Principal [name=user1, type=USER] does not have following privileges for operation QUERY [[DELETE] on Object [type=TABLE_OR_VIEW, name=default.auth_noupd], [INSERT] on Object [type=TABLE_OR_VIEW, name=default.auth_noupd, action=INSERT], [SELECT] on Object [type=TABLE_OR_VIEW, name=default.auth_noupd]]
diff --git a/ql/src/test/results/clientnegative/update_notnull_constraint.q.out b/ql/src/test/results/clientnegative/update_notnull_constraint.q.out
index 86bfc674802..f554da6e624 100644
--- a/ql/src/test/results/clientnegative/update_notnull_constraint.q.out
+++ b/ql/src/test/results/clientnegative/update_notnull_constraint.q.out
@@ -21,4 +21,30 @@ POSTHOOK: Output: default@acid_uami
POSTHOOK: Lineage: acid_uami.de SCRIPT []
POSTHOOK: Lineage: acid_uami.i SCRIPT []
POSTHOOK: Lineage: acid_uami.vc SCRIPT []
-FAILED: DataConstraintViolationError org.apache.hadoop.hive.ql.exec.errors.DataConstraintViolationError: Either CHECK or NOT NULL constraint violated!
+PREHOOK: query: UPDATE acid_uami set de=null where i=1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@acid_uami
+PREHOOK: Output: default@acid_uami
+PREHOOK: Output: default@acid_uami
+Status: Failed
+Vertex failed, vertexName=Map 1, vertexId=vertex_#ID#, diagnostics=[Task failed, taskId=task_#ID#, diagnostics=[TaskAttempt 0 failed, info=[Error: Error while running task ( failure ) : java.lang.RuntimeException: org.apache.hadoop.hive.ql.exec.errors.DataConstraintViolationError: Either CHECK or NOT NULL constraint violated!
+#### A masked pattern was here ####
+Caused by: org.apache.hadoop.hive.ql.exec.errors.DataConstraintViolationError: Either CHECK or NOT NULL constraint violated!
+#### A masked pattern was here ####
+, errorMessage=Cannot recover from this error:java.lang.RuntimeException: org.apache.hadoop.hive.ql.exec.errors.DataConstraintViolationError: Either CHECK or NOT NULL constraint violated!
+#### A masked pattern was here ####
+Caused by: org.apache.hadoop.hive.ql.exec.errors.DataConstraintViolationError: Either CHECK or NOT NULL constraint violated!
+#### A masked pattern was here ####
+]], Vertex did not succeed due to OWN_TASK_FAILURE, failedTasks:1 killedTasks:0, Vertex vertex_#ID# [Map 1] killed/failed due to:OWN_TASK_FAILURE]
+[Masked Vertex killed due to OTHER_VERTEX_FAILURE]
+[Masked Vertex killed due to OTHER_VERTEX_FAILURE]
+DAG did not succeed due to VERTEX_FAILURE. failedVertices:1 killedVertices:2
+FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.tez.TezTask. Vertex failed, vertexName=Map 1, vertexId=vertex_#ID#, diagnostics=[Task failed, taskId=task_#ID#, diagnostics=[TaskAttempt 0 failed, info=[Error: Error while running task ( failure ) : java.lang.RuntimeException: org.apache.hadoop.hive.ql.exec.errors.DataConstraintViolationError: Either CHECK or NOT NULL constraint violated!
+#### A masked pattern was here ####
+Caused by: org.apache.hadoop.hive.ql.exec.errors.DataConstraintViolationError: Either CHECK or NOT NULL constraint violated!
+#### A masked pattern was here ####
+, errorMessage=Cannot recover from this error:java.lang.RuntimeException: org.apache.hadoop.hive.ql.exec.errors.DataConstraintViolationError: Either CHECK or NOT NULL constraint violated!
+#### A masked pattern was here ####
+Caused by: org.apache.hadoop.hive.ql.exec.errors.DataConstraintViolationError: Either CHECK or NOT NULL constraint violated!
+#### A masked pattern was here ####
+]], Vertex did not succeed due to OWN_TASK_FAILURE, failedTasks:1 killedTasks:0, Vertex vertex_#ID# [Map 1] killed/failed due to:OWN_TASK_FAILURE][Masked Vertex killed due to OTHER_VERTEX_FAILURE][Masked Vertex killed due to OTHER_VERTEX_FAILURE]DAG did not succeed due to VERTEX_FAILURE. failedVertices:1 killedVertices:2
diff --git a/ql/src/test/results/clientpositive/llap/acid_direct_update_delete.q.out b/ql/src/test/results/clientpositive/llap/acid_direct_update_delete.q.out
index d71173df87a..4ddda5942ab 100644
--- a/ql/src/test/results/clientpositive/llap/acid_direct_update_delete.q.out
+++ b/ql/src/test/results/clientpositive/llap/acid_direct_update_delete.q.out
@@ -60,10 +60,14 @@ PREHOOK: query: UPDATE test_update_bucketed SET value='New value' WHERE id IN ('
PREHOOK: type: QUERY
PREHOOK: Input: default@test_update_bucketed
PREHOOK: Output: default@test_update_bucketed
+PREHOOK: Output: default@test_update_bucketed
POSTHOOK: query: UPDATE test_update_bucketed SET value='New value' WHERE id IN ('6','11', '18', '20')
POSTHOOK: type: QUERY
POSTHOOK: Input: default@test_update_bucketed
POSTHOOK: Output: default@test_update_bucketed
+POSTHOOK: Output: default@test_update_bucketed
+POSTHOOK: Lineage: test_update_bucketed.id SIMPLE [(test_update_bucketed)test_update_bucketed.FieldSchema(name:id, type:string, comment:null), ]
+POSTHOOK: Lineage: test_update_bucketed.value SIMPLE []
PREHOOK: query: SELECT * FROM test_update_bucketed
PREHOOK: type: QUERY
PREHOOK: Input: default@test_update_bucketed
@@ -100,10 +104,14 @@ PREHOOK: query: UPDATE test_update_bucketed SET value='New value2' WHERE id IN (
PREHOOK: type: QUERY
PREHOOK: Input: default@test_update_bucketed
PREHOOK: Output: default@test_update_bucketed
+PREHOOK: Output: default@test_update_bucketed
POSTHOOK: query: UPDATE test_update_bucketed SET value='New value2' WHERE id IN ('2','18', '19')
POSTHOOK: type: QUERY
POSTHOOK: Input: default@test_update_bucketed
POSTHOOK: Output: default@test_update_bucketed
+POSTHOOK: Output: default@test_update_bucketed
+POSTHOOK: Lineage: test_update_bucketed.id SIMPLE [(test_update_bucketed)test_update_bucketed.FieldSchema(name:id, type:string, comment:null), ]
+POSTHOOK: Lineage: test_update_bucketed.value SIMPLE []
PREHOOK: query: SELECT * FROM test_update_bucketed
PREHOOK: type: QUERY
PREHOOK: Input: default@test_update_bucketed
diff --git a/ql/src/test/results/clientpositive/llap/acid_direct_update_delete_partitions.q.out b/ql/src/test/results/clientpositive/llap/acid_direct_update_delete_partitions.q.out
index ec4ae9e8c71..6b1211ec1e3 100644
--- a/ql/src/test/results/clientpositive/llap/acid_direct_update_delete_partitions.q.out
+++ b/ql/src/test/results/clientpositive/llap/acid_direct_update_delete_partitions.q.out
@@ -93,6 +93,7 @@ PREHOOK: Input: default@test_update_part@c=11
PREHOOK: Input: default@test_update_part@c=22
PREHOOK: Input: default@test_update_part@c=33
PREHOOK: Input: default@test_update_part@c=__HIVE_DEFAULT_PARTITION__
+PREHOOK: Output: default@test_update_part
PREHOOK: Output: default@test_update_part@c=11
PREHOOK: Output: default@test_update_part@c=22
PREHOOK: Output: default@test_update_part@c=33
@@ -104,10 +105,14 @@ POSTHOOK: Input: default@test_update_part@c=11
POSTHOOK: Input: default@test_update_part@c=22
POSTHOOK: Input: default@test_update_part@c=33
POSTHOOK: Input: default@test_update_part@c=__HIVE_DEFAULT_PARTITION__
+POSTHOOK: Output: default@test_update_part
POSTHOOK: Output: default@test_update_part@c=11
POSTHOOK: Output: default@test_update_part@c=22
+POSTHOOK: Output: default@test_update_part@c=22
POSTHOOK: Output: default@test_update_part@c=33
POSTHOOK: Output: default@test_update_part@c=__HIVE_DEFAULT_PARTITION__
+POSTHOOK: Lineage: test_update_part PARTITION(c=22).a SIMPLE [(test_update_part)test_update_part.FieldSchema(name:a, type:int, comment:null), ]
+POSTHOOK: Lineage: test_update_part PARTITION(c=22).b SIMPLE []
PREHOOK: query: UPDATE test_update_part SET b=2222 WHERE a=8
PREHOOK: type: QUERY
PREHOOK: Input: default@test_update_part
@@ -115,6 +120,7 @@ PREHOOK: Input: default@test_update_part@c=11
PREHOOK: Input: default@test_update_part@c=22
PREHOOK: Input: default@test_update_part@c=33
PREHOOK: Input: default@test_update_part@c=__HIVE_DEFAULT_PARTITION__
+PREHOOK: Output: default@test_update_part
PREHOOK: Output: default@test_update_part@c=11
PREHOOK: Output: default@test_update_part@c=22
PREHOOK: Output: default@test_update_part@c=33
@@ -126,6 +132,7 @@ POSTHOOK: Input: default@test_update_part@c=11
POSTHOOK: Input: default@test_update_part@c=22
POSTHOOK: Input: default@test_update_part@c=33
POSTHOOK: Input: default@test_update_part@c=__HIVE_DEFAULT_PARTITION__
+POSTHOOK: Output: default@test_update_part
POSTHOOK: Output: default@test_update_part@c=11
POSTHOOK: Output: default@test_update_part@c=22
POSTHOOK: Output: default@test_update_part@c=33
@@ -137,6 +144,7 @@ PREHOOK: Input: default@test_update_part@c=11
PREHOOK: Input: default@test_update_part@c=22
PREHOOK: Input: default@test_update_part@c=33
PREHOOK: Input: default@test_update_part@c=__HIVE_DEFAULT_PARTITION__
+PREHOOK: Output: default@test_update_part
PREHOOK: Output: default@test_update_part@c=11
PREHOOK: Output: default@test_update_part@c=22
PREHOOK: Output: default@test_update_part@c=33
@@ -148,30 +156,50 @@ POSTHOOK: Input: default@test_update_part@c=11
POSTHOOK: Input: default@test_update_part@c=22
POSTHOOK: Input: default@test_update_part@c=33
POSTHOOK: Input: default@test_update_part@c=__HIVE_DEFAULT_PARTITION__
+POSTHOOK: Output: default@test_update_part
+POSTHOOK: Output: default@test_update_part@c=11
POSTHOOK: Output: default@test_update_part@c=11
POSTHOOK: Output: default@test_update_part@c=22
+POSTHOOK: Output: default@test_update_part@c=22
POSTHOOK: Output: default@test_update_part@c=33
POSTHOOK: Output: default@test_update_part@c=__HIVE_DEFAULT_PARTITION__
+POSTHOOK: Output: default@test_update_part@c=__HIVE_DEFAULT_PARTITION__
+POSTHOOK: Lineage: test_update_part PARTITION(c=11).a SIMPLE []
+POSTHOOK: Lineage: test_update_part PARTITION(c=11).b SIMPLE []
+POSTHOOK: Lineage: test_update_part PARTITION(c=22).a SIMPLE []
+POSTHOOK: Lineage: test_update_part PARTITION(c=22).b SIMPLE []
+POSTHOOK: Lineage: test_update_part PARTITION(c=__HIVE_DEFAULT_PARTITION__).a SIMPLE []
+POSTHOOK: Lineage: test_update_part PARTITION(c=__HIVE_DEFAULT_PARTITION__).b SIMPLE []
PREHOOK: query: UPDATE test_update_part SET b=4444 WHERE c is null
PREHOOK: type: QUERY
PREHOOK: Input: default@test_update_part
PREHOOK: Input: default@test_update_part@c=__HIVE_DEFAULT_PARTITION__
+PREHOOK: Output: default@test_update_part
PREHOOK: Output: default@test_update_part@c=__HIVE_DEFAULT_PARTITION__
POSTHOOK: query: UPDATE test_update_part SET b=4444 WHERE c is null
POSTHOOK: type: QUERY
POSTHOOK: Input: default@test_update_part
POSTHOOK: Input: default@test_update_part@c=__HIVE_DEFAULT_PARTITION__
+POSTHOOK: Output: default@test_update_part
+POSTHOOK: Output: default@test_update_part@c=__HIVE_DEFAULT_PARTITION__
POSTHOOK: Output: default@test_update_part@c=__HIVE_DEFAULT_PARTITION__
+POSTHOOK: Lineage: test_update_part PARTITION(c=__HIVE_DEFAULT_PARTITION__).a SIMPLE [(test_update_part)test_update_part.FieldSchema(name:a, type:int, comment:null), ]
+POSTHOOK: Lineage: test_update_part PARTITION(c=__HIVE_DEFAULT_PARTITION__).b SIMPLE []
PREHOOK: query: UPDATE test_update_part SET b=5555 WHERE c=33
PREHOOK: type: QUERY
PREHOOK: Input: default@test_update_part
PREHOOK: Input: default@test_update_part@c=33
+PREHOOK: Output: default@test_update_part
PREHOOK: Output: default@test_update_part@c=33
POSTHOOK: query: UPDATE test_update_part SET b=5555 WHERE c=33
POSTHOOK: type: QUERY
POSTHOOK: Input: default@test_update_part
POSTHOOK: Input: default@test_update_part@c=33
+POSTHOOK: Output: default@test_update_part
POSTHOOK: Output: default@test_update_part@c=33
+POSTHOOK: Output: default@test_update_part@c=33
+POSTHOOK: Lineage: test_update_part PARTITION(c=33).a SIMPLE [(test_update_part)test_update_part.FieldSchema(name:a, type:int, comment:null), ]
+POSTHOOK: Lineage: test_update_part PARTITION(c=33).b SIMPLE []
PREHOOK: query: UPDATE test_update_part SET b=6666 WHERE a IN (SELECT a FROM test_update_part_text WHERE (c=11 and b=2) or c=33)
PREHOOK: type: QUERY
PREHOOK: Input: default@test_update_part
@@ -180,6 +208,7 @@ PREHOOK: Input: default@test_update_part@c=22
PREHOOK: Input: default@test_update_part@c=33
PREHOOK: Input: default@test_update_part@c=__HIVE_DEFAULT_PARTITION__
PREHOOK: Input: default@test_update_part_text
+PREHOOK: Output: default@test_update_part
PREHOOK: Output: default@test_update_part@c=11
PREHOOK: Output: default@test_update_part@c=22
PREHOOK: Output: default@test_update_part@c=33
@@ -192,10 +221,17 @@ POSTHOOK: Input: default@test_update_part@c=22
POSTHOOK: Input: default@test_update_part@c=33
POSTHOOK: Input: default@test_update_part@c=__HIVE_DEFAULT_PARTITION__
POSTHOOK: Input: default@test_update_part_text
+POSTHOOK: Output: default@test_update_part
+POSTHOOK: Output: default@test_update_part@c=11
POSTHOOK: Output: default@test_update_part@c=11
POSTHOOK: Output: default@test_update_part@c=22
POSTHOOK: Output: default@test_update_part@c=33
+POSTHOOK: Output: default@test_update_part@c=33
POSTHOOK: Output: default@test_update_part@c=__HIVE_DEFAULT_PARTITION__
+POSTHOOK: Lineage: test_update_part PARTITION(c=11).a SIMPLE [(test_update_part)test_update_part.FieldSchema(name:a, type:int, comment:null), ]
+POSTHOOK: Lineage: test_update_part PARTITION(c=11).b SIMPLE []
+POSTHOOK: Lineage: test_update_part PARTITION(c=33).a SIMPLE [(test_update_part)test_update_part.FieldSchema(name:a, type:int, comment:null), ]
+POSTHOOK: Lineage: test_update_part PARTITION(c=33).b SIMPLE []
PREHOOK: query: SELECT * FROM test_update_part ORDER BY c, b, a
PREHOOK: type: QUERY
PREHOOK: Input: default@test_update_part
diff --git a/ql/src/test/results/clientpositive/llap/acid_insert_overwrite_update.q.out b/ql/src/test/results/clientpositive/llap/acid_insert_overwrite_update.q.out
index cbdd6c6cb10..e220dddba64 100644
--- a/ql/src/test/results/clientpositive/llap/acid_insert_overwrite_update.q.out
+++ b/ql/src/test/results/clientpositive/llap/acid_insert_overwrite_update.q.out
@@ -36,10 +36,17 @@ PREHOOK: query: update sequential_update set mtime = current_timestamp, seq=1, p
PREHOOK: type: QUERY
PREHOOK: Input: default@sequential_update
PREHOOK: Output: default@sequential_update
+PREHOOK: Output: default@sequential_update
POSTHOOK: query: update sequential_update set mtime = current_timestamp, seq=1, prev_writeid = ROW__ID.writeId, prev_dirname = regexp_extract(INPUT__FILE__NAME, '.*/(.*)/[^/]*', 1)
POSTHOOK: type: QUERY
POSTHOOK: Input: default@sequential_update
POSTHOOK: Output: default@sequential_update
+POSTHOOK: Output: default@sequential_update
+POSTHOOK: Lineage: sequential_update.ctime SIMPLE [(sequential_update)sequential_update.FieldSchema(name:ctime, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: sequential_update.mtime SIMPLE []
+POSTHOOK: Lineage: sequential_update.prev_dirname EXPRESSION [(sequential_update)sequential_update.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ]
+POSTHOOK: Lineage: sequential_update.prev_writeid EXPRESSION [(sequential_update)sequential_update.FieldSchema(name:ROW__ID, type:struct<writeId:bigint,bucketId:int,rowId:bigint>, comment:), ]
+POSTHOOK: Lineage: sequential_update.seq EXPRESSION []
PREHOOK: query: select distinct IF(seq==1, 'LOOKS OKAY', 'BROKEN'), regexp_extract(INPUT__FILE__NAME, '.*/(.*)/[^/]*', 1) from sequential_update
PREHOOK: type: QUERY
PREHOOK: Input: default@sequential_update
@@ -48,7 +55,7 @@ POSTHOOK: query: select distinct IF(seq==1, 'LOOKS OKAY', 'BROKEN'), regexp_extr
POSTHOOK: type: QUERY
POSTHOOK: Input: default@sequential_update
#### A masked pattern was here ####
-LOOKS OKAY delta_0000002_0000002_0000
+LOOKS OKAY delta_0000002_0000002_0001
PREHOOK: query: insert overwrite table sequential_update values(current_timestamp, 0, current_timestamp, 0, '')
PREHOOK: type: QUERY
PREHOOK: Input: _dummy_database@_dummy_table
@@ -66,10 +73,17 @@ PREHOOK: query: update sequential_update set mtime = current_timestamp, seq=1, p
PREHOOK: type: QUERY
PREHOOK: Input: default@sequential_update
PREHOOK: Output: default@sequential_update
+PREHOOK: Output: default@sequential_update
POSTHOOK: query: update sequential_update set mtime = current_timestamp, seq=1, prev_writeid = ROW__ID.writeId, prev_dirname = regexp_extract(INPUT__FILE__NAME, '.*/(.*)/[^/]*', 1)
POSTHOOK: type: QUERY
POSTHOOK: Input: default@sequential_update
POSTHOOK: Output: default@sequential_update
+POSTHOOK: Output: default@sequential_update
+POSTHOOK: Lineage: sequential_update.ctime SIMPLE [(sequential_update)sequential_update.FieldSchema(name:ctime, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: sequential_update.mtime SIMPLE []
+POSTHOOK: Lineage: sequential_update.prev_dirname EXPRESSION [(sequential_update)sequential_update.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ]
+POSTHOOK: Lineage: sequential_update.prev_writeid EXPRESSION [(sequential_update)sequential_update.FieldSchema(name:ROW__ID, type:struct<writeId:bigint,bucketId:int,rowId:bigint>, comment:), ]
+POSTHOOK: Lineage: sequential_update.seq EXPRESSION []
PREHOOK: query: select distinct IF(seq==1, 'LOOKS OKAY', 'BROKEN'), regexp_extract(INPUT__FILE__NAME, '.*/(.*)/[^/]*', 1) from sequential_update
PREHOOK: type: QUERY
PREHOOK: Input: default@sequential_update
@@ -78,7 +92,7 @@ POSTHOOK: query: select distinct IF(seq==1, 'LOOKS OKAY', 'BROKEN'), regexp_extr
POSTHOOK: type: QUERY
POSTHOOK: Input: default@sequential_update
#### A masked pattern was here ####
-LOOKS OKAY delta_0000004_0000004_0000
+LOOKS OKAY delta_0000004_0000004_0001
PREHOOK: query: alter table sequential_update compact 'major'
PREHOOK: type: ALTERTABLE_COMPACT
POSTHOOK: query: alter table sequential_update compact 'major'
@@ -91,7 +105,7 @@ POSTHOOK: query: select distinct IF(seq==1, 'LOOKS OKAY', 'BROKEN'), regexp_extr
POSTHOOK: type: QUERY
POSTHOOK: Input: default@sequential_update
#### A masked pattern was here ####
-LOOKS OKAY delta_0000004_0000004_0000
+LOOKS OKAY delta_0000004_0000004_0001
PREHOOK: query: insert overwrite table sequential_update values(current_timestamp, 0, current_timestamp, 0, ''), (current_timestamp, 2, current_timestamp, 2, '')
PREHOOK: type: QUERY
PREHOOK: Input: _dummy_database@_dummy_table
@@ -186,6 +200,7 @@ PREHOOK: Input: default@orc_test_txn
PREHOOK: Input: default@orc_test_txn@year=2016
PREHOOK: Input: default@orc_test_txn@year=2017
PREHOOK: Input: default@orc_test_txn@year=2018
+PREHOOK: Output: default@orc_test_txn
PREHOOK: Output: default@orc_test_txn@year=2016
PREHOOK: Output: default@orc_test_txn@year=2017
PREHOOK: Output: default@orc_test_txn@year=2018
@@ -195,9 +210,22 @@ POSTHOOK: Input: default@orc_test_txn
POSTHOOK: Input: default@orc_test_txn@year=2016
POSTHOOK: Input: default@orc_test_txn@year=2017
POSTHOOK: Input: default@orc_test_txn@year=2018
+POSTHOOK: Output: default@orc_test_txn
+POSTHOOK: Output: default@orc_test_txn@year=2016
POSTHOOK: Output: default@orc_test_txn@year=2016
POSTHOOK: Output: default@orc_test_txn@year=2017
+POSTHOOK: Output: default@orc_test_txn@year=2017
+POSTHOOK: Output: default@orc_test_txn@year=2018
POSTHOOK: Output: default@orc_test_txn@year=2018
+POSTHOOK: Lineage: orc_test_txn PARTITION(year=2016).dept SIMPLE [(orc_test_txn)orc_test_txn.FieldSchema(name:dept, type:string, comment:null), ]
+POSTHOOK: Lineage: orc_test_txn PARTITION(year=2016).id SIMPLE []
+POSTHOOK: Lineage: orc_test_txn PARTITION(year=2016).name SIMPLE [(orc_test_txn)orc_test_txn.FieldSchema(name:name, type:string, comment:null), ]
+POSTHOOK: Lineage: orc_test_txn PARTITION(year=2017).dept SIMPLE [(orc_test_txn)orc_test_txn.FieldSchema(name:dept, type:string, comment:null), ]
+POSTHOOK: Lineage: orc_test_txn PARTITION(year=2017).id SIMPLE []
+POSTHOOK: Lineage: orc_test_txn PARTITION(year=2017).name SIMPLE [(orc_test_txn)orc_test_txn.FieldSchema(name:name, type:string, comment:null), ]
+POSTHOOK: Lineage: orc_test_txn PARTITION(year=2018).dept SIMPLE [(orc_test_txn)orc_test_txn.FieldSchema(name:dept, type:string, comment:null), ]
+POSTHOOK: Lineage: orc_test_txn PARTITION(year=2018).id SIMPLE []
+POSTHOOK: Lineage: orc_test_txn PARTITION(year=2018).name SIMPLE [(orc_test_txn)orc_test_txn.FieldSchema(name:name, type:string, comment:null), ]
PREHOOK: query: select distinct IF(id==1, 'LOOKS OKAY', 'BROKEN') from orc_test_txn
PREHOOK: type: QUERY
PREHOOK: Input: default@orc_test_txn
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 a20ae6ed4f0..65f3c3e5873 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
@@ -111,6 +111,7 @@ PREHOOK: type: QUERY
PREHOOK: Input: default@srcpart_acid
PREHOOK: Input: default@srcpart_acid@ds=2008-04-08/hr=11
PREHOOK: Input: default@srcpart_acid@ds=2008-04-09/hr=11
+PREHOOK: Output: default@srcpart_acid
PREHOOK: Output: default@srcpart_acid@ds=2008-04-08/hr=11
PREHOOK: Output: default@srcpart_acid@ds=2008-04-09/hr=11
POSTHOOK: query: explain update srcpart_acid set value = concat(value, 'updated') where cast(key as integer) in(413,43) and hr='11'
@@ -118,16 +119,19 @@ POSTHOOK: type: QUERY
POSTHOOK: Input: default@srcpart_acid
POSTHOOK: Input: default@srcpart_acid@ds=2008-04-08/hr=11
POSTHOOK: Input: default@srcpart_acid@ds=2008-04-09/hr=11
+POSTHOOK: Output: default@srcpart_acid
POSTHOOK: Output: default@srcpart_acid@ds=2008-04-08/hr=11
POSTHOOK: Output: default@srcpart_acid@ds=2008-04-09/hr=11
STAGE DEPENDENCIES:
- Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
- Stage-0 depends on stages: Stage-2
- Stage-3 depends on stages: Stage-0
+ Stage-2 is a root stage
+ Stage-3 depends on stages: Stage-2
+ Stage-0 depends on stages: Stage-3
+ Stage-4 depends on stages: Stage-0
+ Stage-1 depends on stages: Stage-3
+ Stage-5 depends on stages: Stage-1
STAGE PLANS:
- Stage: Stage-1
+ Stage: Stage-2
Tez
#### A masked pattern was here ####
Edges:
@@ -147,33 +151,50 @@ STAGE PLANS:
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
Statistics: Num rows: 500 Data size: 265500 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: 500 Data size: 265500 Basic stats: COMPLETE Column stats: PARTIAL
- value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string)
+ Select Operator
+ expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col3 (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 500 Data size: 130000 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: 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)
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Statistics: Num rows: 500 Data size: 270500 Basic stats: COMPLETE Column stats: PARTIAL
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 500 Data size: 270500 Basic stats: COMPLETE Column stats: PARTIAL
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.srcpart_acid
+ Write Type: INSERT
Execution mode: llap
LLAP IO: may be used (ACID table)
Reducer 2
Execution mode: llap
Reduce Operator Tree:
Select Operator
- expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), '11' (type: string)
- outputColumnNames: _col0, _col1, _col2, _col3, _col4
- Statistics: Num rows: 500 Data size: 308500 Basic stats: COMPLETE Column stats: PARTIAL
+ expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), VALUE._col0 (type: string), '11' (type: string)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 500 Data size: 173000 Basic stats: COMPLETE Column stats: PARTIAL
File Output Operator
compressed: false
- Statistics: Num rows: 500 Data size: 308500 Basic stats: COMPLETE Column stats: PARTIAL
+ Statistics: Num rows: 500 Data size: 173000 Basic stats: COMPLETE Column stats: PARTIAL
table:
input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.srcpart_acid
- Write Type: UPDATE
+ Write Type: DELETE
- Stage: Stage-2
+ Stage: Stage-3
Dependency Collection
Stage: Stage-0
@@ -188,9 +209,27 @@ STAGE PLANS:
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.srcpart_acid
- Write Type: UPDATE
+ Write Type: DELETE
- Stage: Stage-3
+ Stage: Stage-4
+ Stats Work
+ Basic Stats Work:
+
+ Stage: Stage-1
+ Move Operator
+ tables:
+ partition:
+ ds
+ hr
+ replace: false
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.srcpart_acid
+ Write Type: INSERT
+
+ Stage: Stage-5
Stats Work
Basic Stats Work:
@@ -199,6 +238,7 @@ PREHOOK: type: QUERY
PREHOOK: Input: default@srcpart_acid
PREHOOK: Input: default@srcpart_acid@ds=2008-04-08/hr=11
PREHOOK: Input: default@srcpart_acid@ds=2008-04-09/hr=11
+PREHOOK: Output: default@srcpart_acid
PREHOOK: Output: default@srcpart_acid@ds=2008-04-08/hr=11
PREHOOK: Output: default@srcpart_acid@ds=2008-04-09/hr=11
POSTHOOK: query: update srcpart_acid set value = concat(value, 'updated') where cast(key as integer) in(413,43) and hr='11'
@@ -206,8 +246,15 @@ POSTHOOK: type: QUERY
POSTHOOK: Input: default@srcpart_acid
POSTHOOK: Input: default@srcpart_acid@ds=2008-04-08/hr=11
POSTHOOK: Input: default@srcpart_acid@ds=2008-04-09/hr=11
+POSTHOOK: Output: default@srcpart_acid
+POSTHOOK: Output: default@srcpart_acid@ds=2008-04-08/hr=11
POSTHOOK: Output: default@srcpart_acid@ds=2008-04-08/hr=11
POSTHOOK: Output: default@srcpart_acid@ds=2008-04-09/hr=11
+POSTHOOK: Output: default@srcpart_acid@ds=2008-04-09/hr=11
+POSTHOOK: Lineage: srcpart_acid PARTITION(ds=2008-04-08,hr=11).key SIMPLE [(srcpart_acid)srcpart_acid.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: srcpart_acid PARTITION(ds=2008-04-08,hr=11).value EXPRESSION [(srcpart_acid)srcpart_acid.FieldSchema(name:value, type:string, comment:null), ]
+POSTHOOK: Lineage: srcpart_acid PARTITION(ds=2008-04-09,hr=11).key SIMPLE [(srcpart_acid)srcpart_acid.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: srcpart_acid PARTITION(ds=2008-04-09,hr=11).value EXPRESSION [(srcpart_acid)srcpart_acid.FieldSchema(name:value, type:string, comment:null), ]
PREHOOK: query: select ds, hr, key, value from srcpart_acid where value like '%updated' order by ds, hr, cast(key as integer)
PREHOOK: type: QUERY
PREHOOK: Input: default@srcpart_acid
@@ -287,7 +334,7 @@ Table Parameters:
numPartitions 4
numRows 2003
rawDataSize 0
- totalSize 18027
+ totalSize 18043
transactional true
transactional_properties default
#### A masked pattern was here ####
@@ -397,7 +444,7 @@ Table Parameters:
numPartitions 4
numRows 2003
rawDataSize 0
- totalSize 18027
+ totalSize 18043
transactional true
transactional_properties default
#### A masked pattern was here ####
@@ -767,6 +814,7 @@ PREHOOK: type: QUERY
PREHOOK: Input: default@srcpart_acidb
PREHOOK: Input: default@srcpart_acidb@ds=2008-04-08/hr=11
PREHOOK: Input: default@srcpart_acidb@ds=2008-04-09/hr=11
+PREHOOK: Output: default@srcpart_acidb
PREHOOK: Output: default@srcpart_acidb@ds=2008-04-08/hr=11
PREHOOK: Output: default@srcpart_acidb@ds=2008-04-09/hr=11
POSTHOOK: query: explain update srcpart_acidb set value = concat(value, 'updated') where cast(key as integer) in(413,43) and hr='11'
@@ -774,20 +822,24 @@ POSTHOOK: type: QUERY
POSTHOOK: Input: default@srcpart_acidb
POSTHOOK: Input: default@srcpart_acidb@ds=2008-04-08/hr=11
POSTHOOK: Input: default@srcpart_acidb@ds=2008-04-09/hr=11
+POSTHOOK: Output: default@srcpart_acidb
POSTHOOK: Output: default@srcpart_acidb@ds=2008-04-08/hr=11
POSTHOOK: Output: default@srcpart_acidb@ds=2008-04-09/hr=11
STAGE DEPENDENCIES:
- Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
- Stage-0 depends on stages: Stage-2
- Stage-3 depends on stages: Stage-0
+ Stage-2 is a root stage
+ Stage-3 depends on stages: Stage-2
+ Stage-0 depends on stages: Stage-3
+ Stage-4 depends on stages: Stage-0
+ Stage-1 depends on stages: Stage-3
+ Stage-5 depends on stages: Stage-1
STAGE PLANS:
- Stage: Stage-1
+ Stage: Stage-2
Tez
#### A masked pattern was here ####
Edges:
Reducer 2 <- Map 1 (SIMPLE_EDGE)
+ Reducer 3 <- Map 1 (SIMPLE_EDGE)
#### A masked pattern was here ####
Vertices:
Map 1
@@ -803,33 +855,64 @@ STAGE PLANS:
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
Statistics: Num rows: 500 Data size: 265500 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: 500 Data size: 308500 Basic stats: COMPLETE Column stats: PARTIAL
- value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string), '11' (type: string)
+ Select Operator
+ expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col3 (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 500 Data size: 130000 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: 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)
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Statistics: Num rows: 500 Data size: 270500 Basic stats: COMPLETE Column stats: PARTIAL
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ null sort order: a
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
+ Statistics: Num rows: 500 Data size: 270500 Basic stats: COMPLETE Column stats: PARTIAL
+ value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string)
Execution mode: llap
LLAP IO: may be used (ACID table)
Reducer 2
Execution mode: llap
Reduce Operator Tree:
Select Operator
- expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: string)
- outputColumnNames: _col0, _col1, _col2, _col3, _col4
- Statistics: Num rows: 500 Data size: 308500 Basic stats: COMPLETE Column stats: PARTIAL
+ expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), VALUE._col0 (type: string), VALUE._col1 (type: string)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 500 Data size: 173000 Basic stats: COMPLETE Column stats: PARTIAL
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 500 Data size: 173000 Basic stats: COMPLETE Column stats: PARTIAL
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.srcpart_acidb
+ Write Type: DELETE
+ Reducer 3
+ Execution mode: llap
+ Reduce Operator Tree:
+ Select Operator
+ expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string)
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Statistics: Num rows: 500 Data size: 270500 Basic stats: COMPLETE Column stats: PARTIAL
File Output Operator
compressed: false
- Statistics: Num rows: 500 Data size: 308500 Basic stats: COMPLETE Column stats: PARTIAL
+ Statistics: Num rows: 500 Data size: 270500 Basic stats: COMPLETE Column stats: PARTIAL
table:
input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.srcpart_acidb
- Write Type: UPDATE
+ Write Type: INSERT
- Stage: Stage-2
+ Stage: Stage-3
Dependency Collection
Stage: Stage-0
@@ -844,9 +927,27 @@ STAGE PLANS:
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.srcpart_acidb
- Write Type: UPDATE
+ Write Type: DELETE
- Stage: Stage-3
+ Stage: Stage-4
+ Stats Work
+ Basic Stats Work:
+
+ Stage: Stage-1
+ Move Operator
+ tables:
+ partition:
+ ds
+ hr
+ replace: false
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.srcpart_acidb
+ Write Type: INSERT
+
+ Stage: Stage-5
Stats Work
Basic Stats Work:
@@ -855,6 +956,7 @@ PREHOOK: type: QUERY
PREHOOK: Input: default@srcpart_acidb
PREHOOK: Input: default@srcpart_acidb@ds=2008-04-08/hr=11
PREHOOK: Input: default@srcpart_acidb@ds=2008-04-09/hr=11
+PREHOOK: Output: default@srcpart_acidb
PREHOOK: Output: default@srcpart_acidb@ds=2008-04-08/hr=11
PREHOOK: Output: default@srcpart_acidb@ds=2008-04-09/hr=11
POSTHOOK: query: update srcpart_acidb set value = concat(value, 'updated') where cast(key as integer) in(413,43) and hr='11'
@@ -862,8 +964,15 @@ POSTHOOK: type: QUERY
POSTHOOK: Input: default@srcpart_acidb
POSTHOOK: Input: default@srcpart_acidb@ds=2008-04-08/hr=11
POSTHOOK: Input: default@srcpart_acidb@ds=2008-04-09/hr=11
+POSTHOOK: Output: default@srcpart_acidb
+POSTHOOK: Output: default@srcpart_acidb@ds=2008-04-08/hr=11
POSTHOOK: Output: default@srcpart_acidb@ds=2008-04-08/hr=11
POSTHOOK: Output: default@srcpart_acidb@ds=2008-04-09/hr=11
+POSTHOOK: Output: default@srcpart_acidb@ds=2008-04-09/hr=11
+POSTHOOK: Lineage: srcpart_acidb PARTITION(ds=2008-04-08,hr=11).key SIMPLE [(srcpart_acidb)srcpart_acidb.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: srcpart_acidb PARTITION(ds=2008-04-08,hr=11).value EXPRESSION [(srcpart_acidb)srcpart_acidb.FieldSchema(name:value, type:string, comment:null), ]
+POSTHOOK: Lineage: srcpart_acidb PARTITION(ds=2008-04-09,hr=11).key SIMPLE [(srcpart_acidb)srcpart_acidb.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: srcpart_acidb PARTITION(ds=2008-04-09,hr=11).value EXPRESSION [(srcpart_acidb)srcpart_acidb.FieldSchema(name:value, type:string, comment:null), ]
PREHOOK: query: select ds, hr, key, value from srcpart_acidb where value like '%updated' order by ds, hr, cast(key as integer)
PREHOOK: type: QUERY
PREHOOK: Input: default@srcpart_acidb
@@ -1304,6 +1413,7 @@ PREHOOK: type: QUERY
PREHOOK: Input: default@srcpart_acidv
PREHOOK: Input: default@srcpart_acidv@ds=2008-04-08/hr=11
PREHOOK: Input: default@srcpart_acidv@ds=2008-04-09/hr=11
+PREHOOK: Output: default@srcpart_acidv
PREHOOK: Output: default@srcpart_acidv@ds=2008-04-08/hr=11
PREHOOK: Output: default@srcpart_acidv@ds=2008-04-09/hr=11
POSTHOOK: query: explain vectorization only detail
@@ -1312,6 +1422,7 @@ POSTHOOK: type: QUERY
POSTHOOK: Input: default@srcpart_acidv
POSTHOOK: Input: default@srcpart_acidv@ds=2008-04-08/hr=11
POSTHOOK: Input: default@srcpart_acidv@ds=2008-04-09/hr=11
+POSTHOOK: Output: default@srcpart_acidv
POSTHOOK: Output: default@srcpart_acidv@ds=2008-04-08/hr=11
POSTHOOK: Output: default@srcpart_acidv@ds=2008-04-09/hr=11
PLAN VECTORIZATION:
@@ -1319,13 +1430,15 @@ PLAN VECTORIZATION:
enabledConditionsMet: [hive.vectorized.execution.enabled IS true]
STAGE DEPENDENCIES:
- Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
- Stage-0 depends on stages: Stage-2
- Stage-3 depends on stages: Stage-0
+ Stage-2 is a root stage
+ Stage-3 depends on stages: Stage-2
+ Stage-0 depends on stages: Stage-3
+ Stage-4 depends on stages: Stage-0
+ Stage-1 depends on stages: Stage-3
+ Stage-5 depends on stages: Stage-1
STAGE PLANS:
- Stage: Stage-1
+ Stage: Stage-2
Tez
Edges:
Reducer 2 <- Map 1 (SIMPLE_EDGE)
@@ -1344,13 +1457,25 @@ STAGE PLANS:
native: true
projectedOutputColumnNums: [4, 0, 7, 2]
selectExpressions: StringGroupColConcatStringScalar(col 1:string, val updated) -> 7:string
- Reduce Sink Vectorization:
- className: VectorReduceSinkObjectHashOperator
- keyColumns: 4:struct<writeid:bigint,bucketid:int,rowid:bigint>
+ Select Vectorization:
+ className: VectorSelectOperator
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
- partitionColumns: 6:int
- valueColumns: 0:string, 7:string, 2:string
+ projectedOutputColumnNums: [4, 2]
+ Reduce Sink Vectorization:
+ className: VectorReduceSinkObjectHashOperator
+ keyColumns: 4:struct<writeid:bigint,bucketid:int,rowid:bigint>
+ 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
+ partitionColumns: 6:int
+ valueColumns: 2:string
+ Select Vectorization:
+ className: VectorSelectOperator
+ native: true
+ projectedOutputColumnNums: [0, 7, 2, 8]
+ selectExpressions: ConstantVectorExpression(val 11) -> 8:string
+ File Sink Vectorization:
+ className: VectorFileSinkOperator
+ native: false
Execution mode: vectorized, llap
LLAP IO: may be used (ACID table)
Map Vectorization:
@@ -1359,7 +1484,7 @@ STAGE PLANS:
inputFormatFeatureSupport: [DECIMAL_64]
featureSupportInUse: [DECIMAL_64]
inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
- allNative: true
+ allNative: false
usesVectorUDFAdaptor: false
vectorized: true
rowBatchContext:
@@ -1369,7 +1494,7 @@ STAGE PLANS:
neededVirtualColumns: [ROWID]
partitionColumnCount: 2
partitionColumns: ds:string, hr:string
- scratchColumnTypeNames: [bigint, string]
+ scratchColumnTypeNames: [bigint, string, string]
Reducer 2
Execution mode: vectorized, llap
Reduce Vectorization:
@@ -1381,31 +1506,36 @@ STAGE PLANS:
usesVectorUDFAdaptor: false
vectorized: true
rowBatchContext:
- dataColumnCount: 4
- dataColumns: KEY.reducesinkkey0:struct<writeid:bigint,bucketid:int,rowid:bigint>, VALUE._col0:string, VALUE._col1:string, VALUE._col2:string
+ dataColumnCount: 2
+ dataColumns: KEY.reducesinkkey0:struct<writeid:bigint,bucketid:int,rowid:bigint>, VALUE._col0:string
partitionColumnCount: 0
scratchColumnTypeNames: [string]
Reduce Operator Tree:
Select Vectorization:
className: VectorSelectOperator
native: true
- projectedOutputColumnNums: [0, 1, 2, 3, 4]
- selectExpressions: ConstantVectorExpression(val 11) -> 4:string
+ projectedOutputColumnNums: [0, 1, 2]
+ selectExpressions: ConstantVectorExpression(val 11) -> 2:string
File Sink Vectorization:
className: VectorFileSinkOperator
native: false
- Stage: Stage-2
+ Stage: Stage-3
Stage: Stage-0
- Stage: Stage-3
+ Stage: Stage-4
+
+ Stage: Stage-1
+
+ Stage: Stage-5
PREHOOK: query: update srcpart_acidv set value = concat(value, 'updated') where cast(key as integer) in(413,43) and hr='11'
PREHOOK: type: QUERY
PREHOOK: Input: default@srcpart_acidv
PREHOOK: Input: default@srcpart_acidv@ds=2008-04-08/hr=11
PREHOOK: Input: default@srcpart_acidv@ds=2008-04-09/hr=11
+PREHOOK: Output: default@srcpart_acidv
PREHOOK: Output: default@srcpart_acidv@ds=2008-04-08/hr=11
PREHOOK: Output: default@srcpart_acidv@ds=2008-04-09/hr=11
POSTHOOK: query: update srcpart_acidv set value = concat(value, 'updated') where cast(key as integer) in(413,43) and hr='11'
@@ -1413,8 +1543,15 @@ POSTHOOK: type: QUERY
POSTHOOK: Input: default@srcpart_acidv
POSTHOOK: Input: default@srcpart_acidv@ds=2008-04-08/hr=11
POSTHOOK: Input: default@srcpart_acidv@ds=2008-04-09/hr=11
+POSTHOOK: Output: default@srcpart_acidv
POSTHOOK: Output: default@srcpart_acidv@ds=2008-04-08/hr=11
+POSTHOOK: Output: default@srcpart_acidv@ds=2008-04-08/hr=11
+POSTHOOK: Output: default@srcpart_acidv@ds=2008-04-09/hr=11
POSTHOOK: Output: default@srcpart_acidv@ds=2008-04-09/hr=11
+POSTHOOK: Lineage: srcpart_acidv PARTITION(ds=2008-04-08,hr=11).key SIMPLE [(srcpart_acidv)srcpart_acidv.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: srcpart_acidv PARTITION(ds=2008-04-08,hr=11).value EXPRESSION [(srcpart_acidv)srcpart_acidv.FieldSchema(name:value, type:string, comment:null), ]
+POSTHOOK: Lineage: srcpart_acidv PARTITION(ds=2008-04-09,hr=11).key SIMPLE [(srcpart_acidv)srcpart_acidv.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: srcpart_acidv PARTITION(ds=2008-04-09,hr=11).value EXPRESSION [(srcpart_acidv)srcpart_acidv.FieldSchema(name:value, type:string, comment:null), ]
PREHOOK: query: select ds, hr, key, value from srcpart_acidv where value like '%updated' order by ds, hr, cast(key as integer)
PREHOOK: type: QUERY
PREHOOK: Input: default@srcpart_acidv
@@ -2216,6 +2353,7 @@ PREHOOK: type: QUERY
PREHOOK: Input: default@srcpart_acidvb
PREHOOK: Input: default@srcpart_acidvb@ds=2008-04-08/hr=11
PREHOOK: Input: default@srcpart_acidvb@ds=2008-04-09/hr=11
+PREHOOK: Output: default@srcpart_acidvb
PREHOOK: Output: default@srcpart_acidvb@ds=2008-04-08/hr=11
PREHOOK: Output: default@srcpart_acidvb@ds=2008-04-09/hr=11
POSTHOOK: query: explain vectorization only detail
@@ -2224,6 +2362,7 @@ POSTHOOK: type: QUERY
POSTHOOK: Input: default@srcpart_acidvb
POSTHOOK: Input: default@srcpart_acidvb@ds=2008-04-08/hr=11
POSTHOOK: Input: default@srcpart_acidvb@ds=2008-04-09/hr=11
+POSTHOOK: Output: default@srcpart_acidvb
POSTHOOK: Output: default@srcpart_acidvb@ds=2008-04-08/hr=11
POSTHOOK: Output: default@srcpart_acidvb@ds=2008-04-09/hr=11
PLAN VECTORIZATION:
@@ -2231,16 +2370,19 @@ PLAN VECTORIZATION:
enabledConditionsMet: [hive.vectorized.execution.enabled IS true]
STAGE DEPENDENCIES:
- Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
- Stage-0 depends on stages: Stage-2
- Stage-3 depends on stages: Stage-0
+ Stage-2 is a root stage
+ Stage-3 depends on stages: Stage-2
+ Stage-0 depends on stages: Stage-3
+ Stage-4 depends on stages: Stage-0
+ Stage-1 depends on stages: Stage-3
+ Stage-5 depends on stages: Stage-1
STAGE PLANS:
- Stage: Stage-1
+ Stage: Stage-2
Tez
Edges:
Reducer 2 <- Map 1 (SIMPLE_EDGE)
+ Reducer 3 <- Map 1 (SIMPLE_EDGE)
Vertices:
Map 1
Map Operator Tree:
@@ -2256,14 +2398,30 @@ STAGE PLANS:
native: true
projectedOutputColumnNums: [4, 0, 7, 2]
selectExpressions: StringGroupColConcatStringScalar(col 1:string, val updated) -> 7:string
- Reduce Sink Vectorization:
- className: VectorReduceSinkObjectHashOperator
- keyColumns: 4:struct<writeid:bigint,bucketid:int,rowid:bigint>
+ Select Vectorization:
+ className: VectorSelectOperator
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
- partitionColumns: 6:int
- valueColumns: 0:string, 7:string, 2:string, 8:string
- valueExpressions: ConstantVectorExpression(val 11) -> 8:string
+ projectedOutputColumnNums: [4, 2]
+ Reduce Sink Vectorization:
+ className: VectorReduceSinkObjectHashOperator
+ keyColumns: 4:struct<writeid:bigint,bucketid:int,rowid:bigint>
+ 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
+ partitionColumns: 6:int
+ valueColumns: 2:string, 9:string
+ valueExpressions: ConstantVectorExpression(val 11) -> 9:string
+ Select Vectorization:
+ className: VectorSelectOperator
+ native: true
+ projectedOutputColumnNums: [0, 7, 2, 8]
+ selectExpressions: ConstantVectorExpression(val 11) -> 8:string
+ Reduce Sink Vectorization:
+ className: VectorReduceSinkObjectHashOperator
+ keyColumns: 0: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
+ partitionColumns: 0:string
+ valueColumns: 7:string, 2:string, 8:string
Execution mode: vectorized, llap
LLAP IO: may be used (ACID table)
Map Vectorization:
@@ -2282,7 +2440,7 @@ STAGE PLANS:
neededVirtualColumns: [ROWID]
partitionColumnCount: 2
partitionColumns: ds:string, hr:string
- scratchColumnTypeNames: [bigint, string, string]
+ scratchColumnTypeNames: [bigint, string, string, string]
Reducer 2
Execution mode: vectorized, llap
Reduce Vectorization:
@@ -2294,30 +2452,58 @@ STAGE PLANS:
usesVectorUDFAdaptor: false
vectorized: true
rowBatchContext:
- dataColumnCount: 5
- dataColumns: KEY.reducesinkkey0:struct<writeid:bigint,bucketid:int,rowid:bigint>, VALUE._col0:string, VALUE._col1:string, VALUE._col2:string, VALUE._col3:string
+ dataColumnCount: 3
+ dataColumns: KEY.reducesinkkey0:struct<writeid:bigint,bucketid:int,rowid:bigint>, VALUE._col0:string, VALUE._col1:string
+ partitionColumnCount: 0
+ scratchColumnTypeNames: []
+ Reduce Operator Tree:
+ Select Vectorization:
+ className: VectorSelectOperator
+ native: true
+ projectedOutputColumnNums: [0, 1, 2]
+ File Sink Vectorization:
+ className: VectorFileSinkOperator
+ native: false
+ Reducer 3
+ Execution mode: vectorized, llap
+ Reduce Vectorization:
+ enabled: true
+ enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez] IS true
+ reduceColumnNullOrder: a
+ reduceColumnSortOrder: +
+ allNative: false
+ usesVectorUDFAdaptor: false
+ vectorized: true
+ rowBatchContext:
+ dataColumnCount: 4
+ dataColumns: KEY.reducesinkkey0:string, VALUE._col0:string, VALUE._col1:string, VALUE._col2:string
partitionColumnCount: 0
scratchColumnTypeNames: []
Reduce Operator Tree:
Select Vectorization:
className: VectorSelectOperator
native: true
- projectedOutputColumnNums: [0, 1, 2, 3, 4]
+ projectedOutputColumnNums: [0, 1, 2, 3]
File Sink Vectorization:
className: VectorFileSinkOperator
native: false
- Stage: Stage-2
+ Stage: Stage-3
Stage: Stage-0
- Stage: Stage-3
+ Stage: Stage-4
+
+ Stage: Stage-1
+
+ Stage: Stage-5
PREHOOK: query: update srcpart_acidvb set value = concat(value, 'updated') where cast(key as integer) in(413,43) and hr='11'
PREHOOK: type: QUERY
PREHOOK: Input: default@srcpart_acidvb
PREHOOK: Input: default@srcpart_acidvb@ds=2008-04-08/hr=11
PREHOOK: Input: default@srcpart_acidvb@ds=2008-04-09/hr=11
+PREHOOK: Output: default@srcpart_acidvb
PREHOOK: Output: default@srcpart_acidvb@ds=2008-04-08/hr=11
PREHOOK: Output: default@srcpart_acidvb@ds=2008-04-09/hr=11
POSTHOOK: query: update srcpart_acidvb set value = concat(value, 'updated') where cast(key as integer) in(413,43) and hr='11'
@@ -2325,8 +2511,15 @@ POSTHOOK: type: QUERY
POSTHOOK: Input: default@srcpart_acidvb
POSTHOOK: Input: default@srcpart_acidvb@ds=2008-04-08/hr=11
POSTHOOK: Input: default@srcpart_acidvb@ds=2008-04-09/hr=11
+POSTHOOK: Output: default@srcpart_acidvb
POSTHOOK: Output: default@srcpart_acidvb@ds=2008-04-08/hr=11
+POSTHOOK: Output: default@srcpart_acidvb@ds=2008-04-08/hr=11
+POSTHOOK: Output: default@srcpart_acidvb@ds=2008-04-09/hr=11
POSTHOOK: Output: default@srcpart_acidvb@ds=2008-04-09/hr=11
+POSTHOOK: Lineage: srcpart_acidvb PARTITION(ds=2008-04-08,hr=11).key SIMPLE [(srcpart_acidvb)srcpart_acidvb.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: srcpart_acidvb PARTITION(ds=2008-04-08,hr=11).value EXPRESSION [(srcpart_acidvb)srcpart_acidvb.FieldSchema(name:value, type:string, comment:null), ]
+POSTHOOK: Lineage: srcpart_acidvb PARTITION(ds=2008-04-09,hr=11).key SIMPLE [(srcpart_acidvb)srcpart_acidvb.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: srcpart_acidvb PARTITION(ds=2008-04-09,hr=11).value EXPRESSION [(srcpart_acidvb)srcpart_acidvb.FieldSchema(name:value, type:string, comment:null), ]
PREHOOK: query: select ds, hr, key, value from srcpart_acidvb where value like '%updated' order by ds, hr, cast(key as integer)
PREHOOK: type: QUERY
PREHOOK: Input: default@srcpart_acidvb
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 4d9dfd29702..c62ee9a8b44 100644
--- a/ql/src/test/results/clientpositive/llap/acid_subquery.q.out
+++ b/ql/src/test/results/clientpositive/llap/acid_subquery.q.out
@@ -89,6 +89,7 @@ PREHOOK: Input: default@target
PREHOOK: Input: default@target@p=1/q=2
PREHOOK: Input: default@target@p=1/q=3
PREHOOK: Input: default@target@p=2/q=2
+PREHOOK: Output: default@target
PREHOOK: Output: default@target@p=1/q=2
PREHOOK: Output: default@target@p=1/q=3
PREHOOK: Output: default@target@p=2/q=2
@@ -99,9 +100,13 @@ POSTHOOK: Input: default@target
POSTHOOK: Input: default@target@p=1/q=2
POSTHOOK: Input: default@target@p=1/q=3
POSTHOOK: Input: default@target@p=2/q=2
+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=1/q=3
POSTHOOK: Output: default@target@p=2/q=2
+POSTHOOK: Lineage: target PARTITION(p=1,q=3).a SIMPLE [(target)target.FieldSchema(name:a, type:int, comment:null), ]
+POSTHOOK: Lineage: target PARTITION(p=1,q=3).b SIMPLE []
PREHOOK: query: select * from target
PREHOOK: type: QUERY
PREHOOK: Input: default@target
@@ -126,12 +131,18 @@ PREHOOK: Input: default@source
PREHOOK: Input: default@target
PREHOOK: Input: default@target@p=2/q=2
PREHOOK: Output: default@source
+PREHOOK: Output: default@source
POSTHOOK: query: update source set b1 = 1 where p1 in (select t.q from target t where t.p=2)
POSTHOOK: type: QUERY
POSTHOOK: Input: default@source
POSTHOOK: Input: default@target
POSTHOOK: Input: default@target@p=2/q=2
POSTHOOK: Output: default@source
+POSTHOOK: Output: default@source
+POSTHOOK: Lineage: source.a1 SIMPLE [(source)source.FieldSchema(name:a1, type:int, comment:null), ]
+POSTHOOK: Lineage: source.b1 SIMPLE []
+POSTHOOK: Lineage: source.p1 SIMPLE [(source)source.FieldSchema(name:p1, type:int, comment:null), ]
+POSTHOOK: Lineage: source.q1 SIMPLE [(source)source.FieldSchema(name:q1, type:int, comment:null), ]
PREHOOK: query: select * from source
PREHOOK: type: QUERY
PREHOOK: Input: default@source
diff --git a/ql/src/test/results/clientpositive/llap/acid_vectorization.q.out b/ql/src/test/results/clientpositive/llap/acid_vectorization.q.out
index 18ec5980f8b..df04f1a80ed 100644
--- a/ql/src/test/results/clientpositive/llap/acid_vectorization.q.out
+++ b/ql/src/test/results/clientpositive/llap/acid_vectorization.q.out
@@ -30,10 +30,14 @@ PREHOOK: query: update acid_vectorized set b = 'foo' where b = 'bar'
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_vectorized
PREHOOK: Output: default@acid_vectorized
+PREHOOK: Output: default@acid_vectorized
POSTHOOK: query: update acid_vectorized set b = 'foo' where b = 'bar'
POSTHOOK: type: QUERY
POSTHOOK: Input: default@acid_vectorized
POSTHOOK: Output: default@acid_vectorized
+POSTHOOK: Output: default@acid_vectorized
+POSTHOOK: Lineage: acid_vectorized.a SIMPLE [(acid_vectorized)acid_vectorized.FieldSchema(name:a, type:int, comment:null), ]
+POSTHOOK: Lineage: acid_vectorized.b SIMPLE []
PREHOOK: query: delete from acid_vectorized where b = 'foo'
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_vectorized
@@ -92,10 +96,14 @@ PREHOOK: query: update acid_fast_vectorized set b = 'foo' where b = 'bar'
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_fast_vectorized
PREHOOK: Output: default@acid_fast_vectorized
+PREHOOK: Output: default@acid_fast_vectorized
POSTHOOK: query: update acid_fast_vectorized set b = 'foo' where b = 'bar'
POSTHOOK: type: QUERY
POSTHOOK: Input: default@acid_fast_vectorized
POSTHOOK: Output: default@acid_fast_vectorized
+POSTHOOK: Output: default@acid_fast_vectorized
+POSTHOOK: Lineage: acid_fast_vectorized.a SIMPLE [(acid_fast_vectorized)acid_fast_vectorized.FieldSchema(name:a, type:int, comment:null), ]
+POSTHOOK: Lineage: acid_fast_vectorized.b SIMPLE []
PREHOOK: query: delete from acid_fast_vectorized where b = 'foo'
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_fast_vectorized
diff --git a/ql/src/test/results/clientpositive/llap/acid_vectorization_original.q.out b/ql/src/test/results/clientpositive/llap/acid_vectorization_original.q.out
index 52761f93707..28ef711b05c 100644
--- a/ql/src/test/results/clientpositive/llap/acid_vectorization_original.q.out
+++ b/ql/src/test/results/clientpositive/llap/acid_vectorization_original.q.out
@@ -563,22 +563,27 @@ PREHOOK: query: explain update over10k_orc_bucketed set i = 0 where b = 42949673
PREHOOK: type: QUERY
PREHOOK: Input: default@over10k_orc_bucketed
PREHOOK: Output: default@over10k_orc_bucketed
+PREHOOK: Output: default@over10k_orc_bucketed
POSTHOOK: query: explain update over10k_orc_bucketed set i = 0 where b = 4294967363 and t < 100
POSTHOOK: type: QUERY
POSTHOOK: Input: default@over10k_orc_bucketed
POSTHOOK: Output: default@over10k_orc_bucketed
+POSTHOOK: Output: default@over10k_orc_bucketed
STAGE DEPENDENCIES:
- Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
- Stage-0 depends on stages: Stage-2
- Stage-3 depends on stages: Stage-0
+ Stage-2 is a root stage
+ Stage-3 depends on stages: Stage-2
+ Stage-0 depends on stages: Stage-3
+ Stage-4 depends on stages: Stage-0
+ Stage-1 depends on stages: Stage-3
+ Stage-5 depends on stages: Stage-1
STAGE PLANS:
- Stage: Stage-1
+ Stage: Stage-2
Tez
#### A masked pattern was here ####
Edges:
Reducer 2 <- Map 1 (SIMPLE_EDGE)
+ Reducer 3 <- Map 1 (SIMPLE_EDGE)
#### A masked pattern was here ####
Vertices:
Map 1
@@ -594,33 +599,63 @@ STAGE PLANS:
expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), t (type: tinyint), si (type: smallint), f (type: float), d (type: double), bo (type: boolean), s (type: string), ts (type: timestamp), dec (type: decimal(4,2)), bin (type: binary)
outputColumnNames: _col0, _col1, _col2, _col5, _col6, _col7, _col8, _col9, _col10, _col11
Statistics: Num rows: 7 Data size: 2483 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: 7 Data size: 2567 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: _col1 (type: tinyint), _col2 (type: smallint), 0 (type: int), 4294967363L (type: bigint), _col5 (type: float), _col6 (type: double), _col7 (type: boolean), _col8 (type: string), _col9 (type: timestamp), _col10 (type: decimal(4,2)), _col11 (type: binary)
+ Select Operator
+ expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ outputColumnNames: _col0
+ Statistics: Num rows: 7 Data size: 532 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: 7 Data size: 532 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: _col1 (type: tinyint), _col2 (type: smallint), 0 (type: int), 4294967363L (type: bigint), _col5 (type: float), _col6 (type: double), _col7 (type: boolean), _col8 (type: string), _col9 (type: timestamp), _col10 (type: decimal(4,2)), _col11 (type: binary)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
+ Statistics: Num rows: 7 Data size: 2035 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col1 (type: smallint)
+ null sort order: a
+ sort order: +
+ Map-reduce partition columns: _col1 (type: smallint)
+ Statistics: Num rows: 7 Data size: 2035 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col0 (type: tinyint), _col2 (type: int), _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: boolean), _col7 (type: string), _col8 (type: timestamp), _col9 (type: decimal(4,2)), _col10 (type: binary)
Execution mode: vectorized, llap
LLAP IO: may be used (ACID table)
Reducer 2
Execution mode: vectorized, llap
Reduce Operator Tree:
Select Operator
- expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), VALUE._col0 (type: tinyint), VALUE._col1 (type: smallint), VALUE._col2 (type: int), VALUE._col3 (type: bigint), VALUE._col4 (type: float), VALUE._col5 (type: double), VALUE._col6 (type: boolean), VALUE._col7 (type: string), VALUE._col8 (type: timestamp), VALUE._col9 (type: decimal(4,2)), VALUE._col10 (type: binary)
- outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
- Statistics: Num rows: 7 Data size: 2567 Basic stats: COMPLETE Column stats: COMPLETE
+ expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ outputColumnNames: _col0
+ Statistics: Num rows: 7 Data size: 532 Basic stats: COMPLETE Column stats: COMPLETE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 7 Data size: 532 Basic stats: COMPLETE Column stats: COMPLETE
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.over10k_orc_bucketed
+ Write Type: DELETE
+ Reducer 3
+ Execution mode: vectorized, llap
+ Reduce Operator Tree:
+ Select Operator
+ expressions: VALUE._col0 (type: tinyint), KEY.reducesinkkey0 (type: smallint), VALUE._col1 (type: int), VALUE._col2 (type: bigint), VALUE._col3 (type: float), VALUE._col4 (type: double), VALUE._col5 (type: boolean), VALUE._col6 (type: string), VALUE._col7 (type: timestamp), VALUE._col8 (type: decimal(4,2)), VALUE._col9 (type: binary)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
+ Statistics: Num rows: 7 Data size: 2035 Basic stats: COMPLETE Column stats: COMPLETE
File Output Operator
compressed: false
- Statistics: Num rows: 7 Data size: 2567 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 7 Data size: 2035 Basic stats: COMPLETE Column stats: COMPLETE
table:
input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.over10k_orc_bucketed
- Write Type: UPDATE
+ Write Type: INSERT
- Stage: Stage-2
+ Stage: Stage-3
Dependency Collection
Stage: Stage-0
@@ -632,9 +667,24 @@ STAGE PLANS:
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.over10k_orc_bucketed
- Write Type: UPDATE
+ Write Type: DELETE
- Stage: Stage-3
+ Stage: Stage-4
+ Stats Work
+ Basic Stats Work:
+
+ Stage: Stage-1
+ Move Operator
+ tables:
+ replace: false
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.over10k_orc_bucketed
+ Write Type: INSERT
+
+ Stage: Stage-5
Stats Work
Basic Stats Work:
@@ -642,10 +692,23 @@ PREHOOK: query: update over10k_orc_bucketed set i = 0 where b = 4294967363 and t
PREHOOK: type: QUERY
PREHOOK: Input: default@over10k_orc_bucketed
PREHOOK: Output: default@over10k_orc_bucketed
+PREHOOK: Output: default@over10k_orc_bucketed
POSTHOOK: query: update over10k_orc_bucketed set i = 0 where b = 4294967363 and t < 100
POSTHOOK: type: QUERY
POSTHOOK: Input: default@over10k_orc_bucketed
POSTHOOK: Output: default@over10k_orc_bucketed
+POSTHOOK: Output: default@over10k_orc_bucketed
+POSTHOOK: Lineage: over10k_orc_bucketed.b SIMPLE []
+POSTHOOK: Lineage: over10k_orc_bucketed.bin SIMPLE [(over10k_orc_bucketed)over10k_orc_bucketed.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over10k_orc_bucketed.bo SIMPLE [(over10k_orc_bucketed)over10k_orc_bucketed.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over10k_orc_bucketed.d SIMPLE [(over10k_orc_bucketed)over10k_orc_bucketed.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over10k_orc_bucketed.dec SIMPLE [(over10k_orc_bucketed)over10k_orc_bucketed.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over10k_orc_bucketed.f SIMPLE [(over10k_orc_bucketed)over10k_orc_bucketed.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over10k_orc_bucketed.i SIMPLE []
+POSTHOOK: Lineage: over10k_orc_bucketed.s SIMPLE [(over10k_orc_bucketed)over10k_orc_bucketed.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over10k_orc_bucketed.si SIMPLE [(over10k_orc_bucketed)over10k_orc_bucketed.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over10k_orc_bucketed.t SIMPLE [(over10k_orc_bucketed)over10k_orc_bucketed.FieldSchema(name:t, type:tinyint, comment:null), ]
+POSTHOOK: Lineage: over10k_orc_bucketed.ts SIMPLE [(over10k_orc_bucketed)over10k_orc_bucketed.FieldSchema(name:ts, type:timestamp, comment:null), ]
PREHOOK: query: select ROW__ID, t, si, i from over10k_orc_bucketed where b = 4294967363 and t < 100 order by ROW__ID
PREHOOK: type: QUERY
PREHOOK: Input: default@over10k_orc_bucketed
@@ -654,12 +717,12 @@ POSTHOOK: query: select ROW__ID, t, si, i from over10k_orc_bucketed where b = 42
POSTHOOK: type: QUERY
POSTHOOK: Input: default@over10k_orc_bucketed
#### A masked pattern was here ####
-{"writeid":### Masked writeid ###,"bucketid":536870912,"rowid":0} 5 501 0
-{"writeid":### Masked writeid ###,"bucketid":536870912,"rowid":1} 5 501 0
-{"writeid":### Masked writeid ###,"bucketid":536936448,"rowid":0} -3 344 0
-{"writeid":### Masked writeid ###,"bucketid":536936448,"rowid":1} -3 344 0
-{"writeid":### Masked writeid ###,"bucketid":537067520,"rowid":0} 35 463 0
-{"writeid":### Masked writeid ###,"bucketid":537067520,"rowid":1} 35 463 0
+{"writeid":### Masked writeid ###,"bucketid":536870913,"rowid":0} 5 501 0
+{"writeid":### Masked writeid ###,"bucketid":536870913,"rowid":1} 5 501 0
+{"writeid":### Masked writeid ###,"bucketid":536936449,"rowid":0} -3 344 0
+{"writeid":### Masked writeid ###,"bucketid":536936449,"rowid":1} -3 344 0
+{"writeid":### Masked writeid ###,"bucketid":537067521,"rowid":0} 35 463 0
+{"writeid":### Masked writeid ###,"bucketid":537067521,"rowid":1} 35 463 0
PREHOOK: query: explain select ROW__ID, count(*) from over10k_orc_bucketed group by ROW__ID having count(*) > 1
PREHOOK: type: QUERY
PREHOOK: Input: default@over10k_orc_bucketed
diff --git a/ql/src/test/results/clientpositive/llap/acid_view_delete.q.out b/ql/src/test/results/clientpositive/llap/acid_view_delete.q.out
index 73e50d5c418..d3355f316a1 100644
--- a/ql/src/test/results/clientpositive/llap/acid_view_delete.q.out
+++ b/ql/src/test/results/clientpositive/llap/acid_view_delete.q.out
@@ -94,7 +94,7 @@ POSTHOOK: Input: default@mydim
#### A masked pattern was here ####
1 bob 95136 true
2 joe 70068 true
-Warning: Shuffle Join MERGEJOIN[37][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[42][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
PREHOOK: query: update mydim set is_current = false
where mydim.key not in(select kv from updates_staging_view)
PREHOOK: type: QUERY
@@ -102,6 +102,7 @@ PREHOOK: Input: default@mydim
PREHOOK: Input: default@updates_staging_table
PREHOOK: Input: default@updates_staging_view
PREHOOK: Output: default@mydim
+PREHOOK: Output: default@mydim
POSTHOOK: query: update mydim set is_current = false
where mydim.key not in(select kv from updates_staging_view)
POSTHOOK: type: QUERY
@@ -109,6 +110,11 @@ POSTHOOK: Input: default@mydim
POSTHOOK: Input: default@updates_staging_table
POSTHOOK: Input: default@updates_staging_view
POSTHOOK: Output: default@mydim
+POSTHOOK: Output: default@mydim
+POSTHOOK: Lineage: mydim.is_current SIMPLE []
+POSTHOOK: Lineage: mydim.key SIMPLE [(mydim)mydim.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: mydim.name SIMPLE [(mydim)mydim.FieldSchema(name:name, type:string, comment:null), ]
+POSTHOOK: Lineage: mydim.zip SIMPLE [(mydim)mydim.FieldSchema(name:zip, type:string, comment:null), ]
PREHOOK: query: select * from mydim order by key
PREHOOK: type: QUERY
PREHOOK: Input: default@mydim
@@ -126,6 +132,7 @@ PREHOOK: Input: default@mydim
PREHOOK: Input: default@updates_staging_table
PREHOOK: Input: default@updates_staging_view
PREHOOK: Output: default@mydim
+PREHOOK: Output: default@mydim
POSTHOOK: query: update mydim set name = 'Olaf'
where mydim.key in(select kv from updates_staging_view)
POSTHOOK: type: QUERY
@@ -133,6 +140,11 @@ POSTHOOK: Input: default@mydim
POSTHOOK: Input: default@updates_staging_table
POSTHOOK: Input: default@updates_staging_view
POSTHOOK: Output: default@mydim
+POSTHOOK: Output: default@mydim
+POSTHOOK: Lineage: mydim.is_current SIMPLE [(mydim)mydim.FieldSchema(name:is_current, type:boolean, comment:null), ]
+POSTHOOK: Lineage: mydim.key SIMPLE [(mydim)mydim.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: mydim.name SIMPLE []
+POSTHOOK: Lineage: mydim.zip SIMPLE [(mydim)mydim.FieldSchema(name:zip, type:string, comment:null), ]
PREHOOK: query: select * from mydim order by key
PREHOOK: type: QUERY
PREHOOK: Input: default@mydim
diff --git a/ql/src/test/results/clientpositive/llap/authorization_update.q.out b/ql/src/test/results/clientpositive/llap/authorization_update.q.out
index a22641ee338..ffc78940ea3 100644
--- a/ql/src/test/results/clientpositive/llap/authorization_update.q.out
+++ b/ql/src/test/results/clientpositive/llap/authorization_update.q.out
@@ -50,7 +50,11 @@ PREHOOK: query: update t_auth_up set j = 0 where i > 0
PREHOOK: type: QUERY
PREHOOK: Input: default@t_auth_up
PREHOOK: Output: default@t_auth_up
+PREHOOK: Output: default@t_auth_up
POSTHOOK: query: update t_auth_up set j = 0 where i > 0
POSTHOOK: type: QUERY
POSTHOOK: Input: default@t_auth_up
POSTHOOK: Output: default@t_auth_up
+POSTHOOK: Output: default@t_auth_up
+POSTHOOK: Lineage: t_auth_up.i SIMPLE [(t_auth_up)t_auth_up.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: t_auth_up.j SIMPLE []
diff --git a/ql/src/test/results/clientpositive/llap/authorization_update_own_table.q.out b/ql/src/test/results/clientpositive/llap/authorization_update_own_table.q.out
index a39640917e0..19f7939a8d1 100644
--- a/ql/src/test/results/clientpositive/llap/authorization_update_own_table.q.out
+++ b/ql/src/test/results/clientpositive/llap/authorization_update_own_table.q.out
@@ -10,10 +10,14 @@ PREHOOK: query: update auth_noupd_n0 set j = 0 where i > 0
PREHOOK: type: QUERY
PREHOOK: Input: default@auth_noupd_n0
PREHOOK: Output: default@auth_noupd_n0
+PREHOOK: Output: default@auth_noupd_n0
POSTHOOK: query: update auth_noupd_n0 set j = 0 where i > 0
POSTHOOK: type: QUERY
POSTHOOK: Input: default@auth_noupd_n0
POSTHOOK: Output: default@auth_noupd_n0
+POSTHOOK: Output: default@auth_noupd_n0
+POSTHOOK: Lineage: auth_noupd_n0.i SIMPLE [(auth_noupd_n0)auth_noupd_n0.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: auth_noupd_n0.j SIMPLE []
PREHOOK: query: set role admin
PREHOOK: type: SHOW_ROLES
POSTHOOK: query: set role admin
diff --git a/ql/src/test/results/clientpositive/llap/bucket_num_reducers_acid.q.out b/ql/src/test/results/clientpositive/llap/bucket_num_reducers_acid.q.out
index 81ba5aec054..1cded294db2 100644
--- a/ql/src/test/results/clientpositive/llap/bucket_num_reducers_acid.q.out
+++ b/ql/src/test/results/clientpositive/llap/bucket_num_reducers_acid.q.out
@@ -22,6 +22,7 @@ PREHOOK: query: update bucket_nr_acid set b = -1
PREHOOK: type: QUERY
PREHOOK: Input: default@bucket_nr_acid
PREHOOK: Output: default@bucket_nr_acid
+PREHOOK: Output: default@bucket_nr_acid
PREHOOK: query: select * from bucket_nr_acid order by a, b
PREHOOK: type: QUERY
PREHOOK: Input: default@bucket_nr_acid
diff --git a/ql/src/test/results/clientpositive/llap/bucket_num_reducers_acid2.q.out b/ql/src/test/results/clientpositive/llap/bucket_num_reducers_acid2.q.out
index 8a4b54aab07..284c0f6097f 100644
--- a/ql/src/test/results/clientpositive/llap/bucket_num_reducers_acid2.q.out
+++ b/ql/src/test/results/clientpositive/llap/bucket_num_reducers_acid2.q.out
@@ -54,10 +54,14 @@ PREHOOK: query: update bucket_nr_acid2 set b = -1
PREHOOK: type: QUERY
PREHOOK: Input: default@bucket_nr_acid2
PREHOOK: Output: default@bucket_nr_acid2
+PREHOOK: Output: default@bucket_nr_acid2
POSTHOOK: query: update bucket_nr_acid2 set b = -1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@bucket_nr_acid2
POSTHOOK: Output: default@bucket_nr_acid2
+POSTHOOK: Output: default@bucket_nr_acid2
+POSTHOOK: Lineage: bucket_nr_acid2.a SIMPLE [(bucket_nr_acid2)bucket_nr_acid2.FieldSchema(name:a, type:int, comment:null), ]
+POSTHOOK: Lineage: bucket_nr_acid2.b SIMPLE []
PREHOOK: query: select * from bucket_nr_acid2 order by a, b
PREHOOK: type: QUERY
PREHOOK: Input: default@bucket_nr_acid2
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 2ab5238e451..a2549978188 100644
--- a/ql/src/test/results/clientpositive/llap/check_constraint.q.out
+++ b/ql/src/test/results/clientpositive/llap/check_constraint.q.out
@@ -2061,65 +2061,87 @@ PREHOOK: query: explain cbo update acid_uami_n0 set de = 893.14 where de = 103.0
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_uami_n0
PREHOOK: Output: default@acid_uami_n0
+PREHOOK: Output: default@acid_uami_n0
POSTHOOK: query: explain cbo update acid_uami_n0 set de = 893.14 where de = 103.00 or de = 119.00
POSTHOOK: type: QUERY
POSTHOOK: Input: default@acid_uami_n0
POSTHOOK: Output: default@acid_uami_n0
+POSTHOOK: Output: default@acid_uami_n0
CBO PLAN:
-HiveSortExchange(distribution=[any], collation=[[0]])
- HiveProject(row__id=[$5], i=[$0], _o__c2=[893.14:DECIMAL(5, 2)], vc=[$2])
- HiveFilter(condition=[AND(IN($1, 103:DECIMAL(3, 0), 119:DECIMAL(3, 0)), enforce_constraint(IS NOT FALSE(>=(893.14, CAST($0):DECIMAL(5, 2)))))])
- HiveTableScan(table=[[default, acid_uami_n0]], table:alias=[acid_uami_n0])
+HiveProject(row__id=[$5], i=[$0], de=[893.14:DECIMAL(5, 2)], vc=[$2])
+ HiveFilter(condition=[IN($1, 103:DECIMAL(3, 0), 119:DECIMAL(3, 0))])
+ HiveTableScan(table=[[default, acid_uami_n0]], table:alias=[acid_uami_n0])
PREHOOK: query: explain update acid_uami_n0 set de = 893.14 where de = 103.00 or de = 119.00
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_uami_n0
PREHOOK: Output: default@acid_uami_n0
+PREHOOK: Output: default@acid_uami_n0
POSTHOOK: query: explain update acid_uami_n0 set de = 893.14 where de = 103.00 or de = 119.00
POSTHOOK: type: QUERY
POSTHOOK: Input: default@acid_uami_n0
POSTHOOK: Output: default@acid_uami_n0
+POSTHOOK: Output: default@acid_uami_n0
STAGE DEPENDENCIES:
- Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
- Stage-0 depends on stages: Stage-2
+ Stage-2 is a root stage
+ Stage-3 depends on stages: Stage-2
+ Stage-0 depends on stages: Stage-3
+ Stage-1 depends on stages: Stage-3
STAGE PLANS:
- Stage: Stage-1
+ Stage: Stage-2
Tez
#### A masked pattern was here ####
Edges:
Reducer 2 <- Map 1 (SIMPLE_EDGE)
+ Reducer 3 <- Map 1 (SIMPLE_EDGE)
#### A masked pattern was here ####
Vertices:
Map 1
Map Operator Tree:
TableScan
alias: acid_uami_n0
- filterExpr: ((de) IN (103, 119) and enforce_constraint((CAST( i AS decimal(5,2)) <= 893.14) is not false)) (type: boolean)
+ filterExpr: (de) IN (103, 119) (type: boolean)
Statistics: Num rows: 1 Data size: 328 Basic stats: COMPLETE Column stats: NONE
Filter Operator
- predicate: ((de) IN (103, 119) and enforce_constraint((CAST( i AS decimal(5,2)) <= 893.14) is not false)) (type: boolean)
+ predicate: (de) IN (103, 119) (type: boolean)
Statistics: Num rows: 1 Data size: 328 Basic stats: COMPLETE Column stats: NONE
Select Operator
expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), i (type: int), vc (type: varchar(128))
outputColumnNames: _col0, _col1, _col3
Statistics: Num rows: 1 Data size: 328 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)
+ Select Operator
+ expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ outputColumnNames: _col0
Statistics: Num rows: 1 Data size: 328 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col1 (type: int), 893.14 (type: decimal(5,2)), _col3 (type: varchar(128))
+ 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: 328 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col1 (type: int), 893.14 (type: decimal(5,2)), _col3 (type: varchar(128))
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 1 Data size: 328 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: enforce_constraint((_col1 is not null and (893.14 >= CAST( _col0 AS decimal(5,2))) is not false)) (type: boolean)
+ Statistics: Num rows: 1 Data size: 328 Basic stats: COMPLETE Column stats: NONE
+ 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: 328 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col1 (type: decimal(5,2)), _col2 (type: varchar(128))
Execution mode: vectorized, llap
LLAP IO: may be used (ACID table)
Reducer 2
Execution mode: vectorized, llap
Reduce Operator Tree:
Select Operator
- expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), VALUE._col0 (type: int), VALUE._col1 (type: decimal(5,2)), VALUE._col2 (type: varchar(128))
- outputColumnNames: _col0, _col1, _col2, _col3
+ expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ outputColumnNames: _col0
Statistics: Num rows: 1 Data size: 328 Basic stats: COMPLETE Column stats: NONE
File Output Operator
compressed: false
@@ -2129,9 +2151,25 @@ STAGE PLANS:
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.acid_uami_n0
- Write Type: UPDATE
+ Write Type: DELETE
+ Reducer 3
+ Execution mode: vectorized, llap
+ Reduce Operator Tree:
+ Select Operator
+ expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: decimal(5,2)), VALUE._col1 (type: varchar(128))
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 1 Data size: 328 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 328 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.acid_uami_n0
+ Write Type: INSERT
- Stage: Stage-2
+ Stage: Stage-3
Dependency Collection
Stage: Stage-0
@@ -2143,16 +2181,32 @@ STAGE PLANS:
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.acid_uami_n0
- Write Type: UPDATE
+ Write Type: DELETE
+
+ Stage: Stage-1
+ Move Operator
+ tables:
+ replace: false
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.acid_uami_n0
+ Write Type: INSERT
PREHOOK: query: update acid_uami_n0 set de = 893.14 where de = 103.00 or de = 119.00
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_uami_n0
PREHOOK: Output: default@acid_uami_n0
+PREHOOK: Output: default@acid_uami_n0
POSTHOOK: query: update acid_uami_n0 set de = 893.14 where de = 103.00 or de = 119.00
POSTHOOK: type: QUERY
POSTHOOK: Input: default@acid_uami_n0
POSTHOOK: Output: default@acid_uami_n0
+POSTHOOK: Output: default@acid_uami_n0
+POSTHOOK: Lineage: acid_uami_n0.de SIMPLE []
+POSTHOOK: Lineage: acid_uami_n0.i SIMPLE [(acid_uami_n0)acid_uami_n0.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: acid_uami_n0.vc SIMPLE [(acid_uami_n0)acid_uami_n0.FieldSchema(name:vc, type:varchar(128), comment:null), ]
PREHOOK: query: select * from acid_uami_n0 order by de desc limit 15
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_uami_n0
@@ -2190,21 +2244,25 @@ PREHOOK: query: explain update acid_uami_n0 set vc = 'apache_hive' where de = 89
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_uami_n0
PREHOOK: Output: default@acid_uami_n0
+PREHOOK: Output: default@acid_uami_n0
POSTHOOK: query: explain update acid_uami_n0 set vc = 'apache_hive' where de = 893.14
POSTHOOK: type: QUERY
POSTHOOK: Input: default@acid_uami_n0
POSTHOOK: Output: default@acid_uami_n0
+POSTHOOK: Output: default@acid_uami_n0
STAGE DEPENDENCIES:
- Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
- Stage-0 depends on stages: Stage-2
+ Stage-2 is a root stage
+ Stage-3 depends on stages: Stage-2
+ Stage-0 depends on stages: Stage-3
+ Stage-1 depends on stages: Stage-3
STAGE PLANS:
- Stage: Stage-1
+ Stage: Stage-2
Tez
#### A masked pattern was here ####
Edges:
Reducer 2 <- Map 1 (SIMPLE_EDGE)
+ Reducer 3 <- Map 1 (SIMPLE_EDGE)
#### A masked pattern was here ####
Vertices:
Map 1
@@ -2220,21 +2278,42 @@ STAGE PLANS:
expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), i (type: int)
outputColumnNames: _col0, _col1
Statistics: Num rows: 1 Data size: 116 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)
+ Select Operator
+ expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ outputColumnNames: _col0
Statistics: Num rows: 1 Data size: 116 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col1 (type: int), 893.14 (type: decimal(5,2)), 'apache_hive' (type: varchar(128))
+ 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: 116 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col1 (type: int), 893.14 (type: decimal(5,2)), 'apache_hive' (type: string)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 1 Data size: 116 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: enforce_constraint(_col1 is not null) (type: boolean)
+ Statistics: Num rows: 1 Data size: 116 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: int), _col1 (type: decimal(5,2)), CAST( _col2 AS varchar(128)) (type: varchar(128))
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 1 Data size: 116 Basic stats: COMPLETE Column stats: NONE
+ 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: 116 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col1 (type: decimal(5,2)), _col2 (type: varchar(128))
Execution mode: vectorized, llap
LLAP IO: may be used (ACID table)
Reducer 2
Execution mode: vectorized, llap
Reduce Operator Tree:
Select Operator
- expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), VALUE._col0 (type: int), VALUE._col1 (type: decimal(5,2)), VALUE._col2 (type: varchar(128))
- outputColumnNames: _col0, _col1, _col2, _col3
+ expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ outputColumnNames: _col0
Statistics: Num rows: 1 Data size: 116 Basic stats: COMPLETE Column stats: NONE
File Output Operator
compressed: false
@@ -2244,9 +2323,25 @@ STAGE PLANS:
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.acid_uami_n0
- Write Type: UPDATE
+ Write Type: DELETE
+ Reducer 3
+ Execution mode: vectorized, llap
+ Reduce Operator Tree:
+ Select Operator
+ expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: decimal(5,2)), VALUE._col1 (type: varchar(128))
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 1 Data size: 116 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 116 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.acid_uami_n0
+ Write Type: INSERT
- Stage: Stage-2
+ Stage: Stage-3
Dependency Collection
Stage: Stage-0
@@ -2258,16 +2353,32 @@ STAGE PLANS:
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.acid_uami_n0
- Write Type: UPDATE
+ Write Type: DELETE
+
+ Stage: Stage-1
+ Move Operator
+ tables:
+ replace: false
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.acid_uami_n0
+ Write Type: INSERT
PREHOOK: query: update acid_uami_n0 set vc = 'apache_hive' where de = 893.14
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_uami_n0
PREHOOK: Output: default@acid_uami_n0
+PREHOOK: Output: default@acid_uami_n0
POSTHOOK: query: update acid_uami_n0 set vc = 'apache_hive' where de = 893.14
POSTHOOK: type: QUERY
POSTHOOK: Input: default@acid_uami_n0
POSTHOOK: Output: default@acid_uami_n0
+POSTHOOK: Output: default@acid_uami_n0
+POSTHOOK: Lineage: acid_uami_n0.de SIMPLE []
+POSTHOOK: Lineage: acid_uami_n0.i SIMPLE [(acid_uami_n0)acid_uami_n0.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: acid_uami_n0.vc EXPRESSION []
PREHOOK: query: select * from acid_uami_n0 order by vc limit 15
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_uami_n0
diff --git a/ql/src/test/results/clientpositive/llap/constant_prop_timestamp_date_cast.q.out b/ql/src/test/results/clientpositive/llap/constant_prop_timestamp_date_cast.q.out
index 765d903f853..3bf0bec722e 100644
--- a/ql/src/test/results/clientpositive/llap/constant_prop_timestamp_date_cast.q.out
+++ b/ql/src/test/results/clientpositive/llap/constant_prop_timestamp_date_cast.q.out
@@ -10,18 +10,22 @@ PREHOOK: query: explain UPDATE constant_prop SET stop_dt = CURRENT_TIMESTAMP WHE
PREHOOK: type: QUERY
PREHOOK: Input: default@constant_prop
PREHOOK: Output: default@constant_prop
+PREHOOK: Output: default@constant_prop
POSTHOOK: query: explain UPDATE constant_prop SET stop_dt = CURRENT_TIMESTAMP WHERE CAST(start_dt AS DATE) = CURRENT_DATE
POSTHOOK: type: QUERY
POSTHOOK: Input: default@constant_prop
POSTHOOK: Output: default@constant_prop
+POSTHOOK: Output: default@constant_prop
STAGE DEPENDENCIES:
- Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
- Stage-0 depends on stages: Stage-2
- Stage-3 depends on stages: Stage-0
+ Stage-2 is a root stage
+ Stage-3 depends on stages: Stage-2
+ Stage-0 depends on stages: Stage-3
+ Stage-4 depends on stages: Stage-0
+ Stage-1 depends on stages: Stage-3
+ Stage-5 depends on stages: Stage-1
STAGE PLANS:
- Stage: Stage-1
+ Stage: Stage-2
Tez
#### A masked pattern was here ####
Edges:
@@ -41,21 +45,37 @@ STAGE PLANS:
expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), start_dt (type: timestamp)
outputColumnNames: _col0, _col1
Statistics: Num rows: 1 Data size: 40 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)
+ Select Operator
+ expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 40 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)
+ Statistics: Num rows: 1 Data size: 40 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col1 (type: timestamp), TIMESTAMP'2020-03-05 14:16:57' (type: timestamp)
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 1 Data size: 40 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col1 (type: timestamp)
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 40 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.constant_prop
+ Write Type: INSERT
Execution mode: vectorized, llap
LLAP IO: may be used (ACID table)
Reducer 2
Execution mode: vectorized, llap
Reduce Operator Tree:
Select Operator
- expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), VALUE._col0 (type: timestamp), TIMESTAMP'2020-03-05 14:16:57' (type: timestamp)
- outputColumnNames: _col0, _col1, _col2
+ expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ outputColumnNames: _col0
Statistics: Num rows: 1 Data size: 40 Basic stats: COMPLETE Column stats: NONE
File Output Operator
compressed: false
@@ -65,9 +85,9 @@ STAGE PLANS:
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.constant_prop
- Write Type: UPDATE
+ Write Type: DELETE
- Stage: Stage-2
+ Stage: Stage-3
Dependency Collection
Stage: Stage-0
@@ -79,9 +99,24 @@ STAGE PLANS:
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.constant_prop
- Write Type: UPDATE
+ Write Type: DELETE
- Stage: Stage-3
+ Stage: Stage-4
+ Stats Work
+ Basic Stats Work:
+
+ Stage: Stage-1
+ Move Operator
+ tables:
+ replace: false
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.constant_prop
+ Write Type: INSERT
+
+ Stage: Stage-5
Stats Work
Basic Stats Work:
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 d8adb179f8c..77673d15e55 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
@@ -120,6 +120,7 @@ PREHOOK: Input: default@target
PREHOOK: Input: default@target@p=1/q=2
PREHOOK: Input: default@target@p=1/q=3
PREHOOK: Input: default@target@p=2/q=2
+PREHOOK: Output: default@target
PREHOOK: Output: default@target@p=1/q=2
PREHOOK: Output: default@target@p=1/q=3
PREHOOK: Output: default@target@p=2/q=2
@@ -130,6 +131,7 @@ POSTHOOK: Input: default@target
POSTHOOK: Input: default@target@p=1/q=2
POSTHOOK: Input: default@target@p=1/q=3
POSTHOOK: Input: default@target@p=2/q=2
+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
@@ -139,12 +141,18 @@ PREHOOK: Input: default@source
PREHOOK: Input: default@target
PREHOOK: Input: default@target@p=2/q=2
PREHOOK: Output: default@source
+PREHOOK: Output: default@source
POSTHOOK: query: update source set b1 = 1 where p1 in (select t.q from target t where t.p=2)
POSTHOOK: type: QUERY
POSTHOOK: Input: default@source
POSTHOOK: Input: default@target
POSTHOOK: Input: default@target@p=2/q=2
POSTHOOK: Output: default@source
+POSTHOOK: Output: default@source
+POSTHOOK: Lineage: source.a1 SIMPLE [(source)source.FieldSchema(name:a1, type:int, comment:null), ]
+POSTHOOK: Lineage: source.b1 SIMPLE []
+POSTHOOK: Lineage: source.p1 SIMPLE [(source)source.FieldSchema(name:p1, type:int, comment:null), ]
+POSTHOOK: Lineage: source.q1 SIMPLE [(source)source.FieldSchema(name:q1, type:int, comment:null), ]
PREHOOK: query: delete from target where p in (select t.q1 from source t where t.a1 = 5)
PREHOOK: type: QUERY
PREHOOK: Input: default@source
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 da2793785a8..90da25badfc 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
@@ -77,24 +77,29 @@ PREHOOK: query: explain update acid_part set value = 'bar' where key = 'foo' and
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_part
PREHOOK: Input: default@acid_part@ds=2008-04-08
+PREHOOK: Output: default@acid_part
PREHOOK: Output: default@acid_part@ds=2008-04-08
POSTHOOK: query: explain update acid_part set value = 'bar' where key = 'foo' and ds='2008-04-08'
POSTHOOK: type: QUERY
POSTHOOK: Input: default@acid_part
POSTHOOK: Input: default@acid_part@ds=2008-04-08
+POSTHOOK: Output: default@acid_part
POSTHOOK: Output: default@acid_part@ds=2008-04-08
STAGE DEPENDENCIES:
- Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
- Stage-0 depends on stages: Stage-2
- Stage-3 depends on stages: Stage-0
+ Stage-2 is a root stage
+ Stage-3 depends on stages: Stage-2
+ Stage-0 depends on stages: Stage-3
+ Stage-4 depends on stages: Stage-0
+ Stage-1 depends on stages: Stage-3
+ Stage-5 depends on stages: Stage-1
STAGE PLANS:
- Stage: Stage-1
+ Stage: Stage-2
Tez
#### A masked pattern was here ####
Edges:
Reducer 2 <- Map 1 (SIMPLE_EDGE)
+ Reducer 3 <- Map 1 (SIMPLE_EDGE)
#### A masked pattern was here ####
Vertices:
Map 1
@@ -115,28 +120,55 @@ STAGE PLANS:
null sort order: z
sort order: +
Map-reduce partition columns: UDFToInteger(_col0) (type: int)
- Statistics: Num rows: 5 Data size: 1720 Basic stats: COMPLETE Column stats: PARTIAL
- value expressions: 'foo' (type: string), 'bar' (type: string), '2008-04-08' (type: string)
+ Statistics: Num rows: 5 Data size: 850 Basic stats: COMPLETE Column stats: PARTIAL
+ value expressions: '2008-04-08' (type: string)
+ Select Operator
+ expressions: 'foo' (type: string), 'bar' (type: string), '2008-04-08' (type: string)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 5 Data size: 1340 Basic stats: COMPLETE Column stats: PARTIAL
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ null sort order: a
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
+ Statistics: Num rows: 5 Data size: 1340 Basic stats: COMPLETE Column stats: PARTIAL
+ value expressions: _col1 (type: string), _col2 (type: string)
Execution mode: llap
LLAP IO: may be used (ACID table)
Reducer 2
Execution mode: llap
Reduce Operator Tree:
Select Operator
- expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string)
- outputColumnNames: _col0, _col1, _col2, _col3
- Statistics: Num rows: 5 Data size: 1720 Basic stats: COMPLETE Column stats: PARTIAL
+ expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), VALUE._col0 (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 5 Data size: 850 Basic stats: COMPLETE Column stats: PARTIAL
File Output Operator
compressed: false
- Statistics: Num rows: 5 Data size: 1720 Basic stats: COMPLETE Column stats: PARTIAL
+ Statistics: Num rows: 5 Data size: 850 Basic stats: COMPLETE Column stats: PARTIAL
table:
input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.acid_part
- Write Type: UPDATE
+ Write Type: DELETE
+ Reducer 3
+ Execution mode: llap
+ Reduce Operator Tree:
+ Select Operator
+ expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string), VALUE._col1 (type: string)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 5 Data size: 1340 Basic stats: COMPLETE Column stats: PARTIAL
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 5 Data size: 1340 Basic stats: COMPLETE Column stats: PARTIAL
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.acid_part
+ Write Type: INSERT
- Stage: Stage-2
+ Stage: Stage-3
Dependency Collection
Stage: Stage-0
@@ -150,9 +182,26 @@ STAGE PLANS:
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.acid_part
- Write Type: UPDATE
+ Write Type: DELETE
- Stage: Stage-3
+ Stage: Stage-4
+ Stats Work
+ Basic Stats Work:
+
+ Stage: Stage-1
+ Move Operator
+ tables:
+ partition:
+ ds
+ replace: false
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.acid_part
+ Write Type: INSERT
+
+ Stage: Stage-5
Stats Work
Basic Stats Work:
@@ -160,12 +209,17 @@ PREHOOK: query: update acid_part set value = 'bar' where key = 'foo' and ds='200
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_part
PREHOOK: Input: default@acid_part@ds=2008-04-08
+PREHOOK: Output: default@acid_part
PREHOOK: Output: default@acid_part@ds=2008-04-08
POSTHOOK: query: update acid_part set value = 'bar' where key = 'foo' and ds='2008-04-08'
POSTHOOK: type: QUERY
POSTHOOK: Input: default@acid_part
POSTHOOK: Input: default@acid_part@ds=2008-04-08
+POSTHOOK: Output: default@acid_part
+POSTHOOK: Output: default@acid_part@ds=2008-04-08
POSTHOOK: Output: default@acid_part@ds=2008-04-08
+POSTHOOK: Lineage: acid_part PARTITION(ds=2008-04-08).key SIMPLE []
+POSTHOOK: Lineage: acid_part PARTITION(ds=2008-04-08).value SIMPLE []
PREHOOK: query: select count(*) from acid_part where ds='2008-04-08'
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_part
@@ -181,24 +235,29 @@ PREHOOK: query: explain update acid_part set value = 'bar' where key = 'foo' and
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_part
PREHOOK: Input: default@acid_part@ds=2008-04-08
+PREHOOK: Output: default@acid_part
PREHOOK: Output: default@acid_part@ds=2008-04-08
POSTHOOK: query: explain update acid_part set value = 'bar' where key = 'foo' and ds in ('2008-04-08')
POSTHOOK: type: QUERY
POSTHOOK: Input: default@acid_part
POSTHOOK: Input: default@acid_part@ds=2008-04-08
+POSTHOOK: Output: default@acid_part
POSTHOOK: Output: default@acid_part@ds=2008-04-08
STAGE DEPENDENCIES:
- Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
- Stage-0 depends on stages: Stage-2
- Stage-3 depends on stages: Stage-0
+ Stage-2 is a root stage
+ Stage-3 depends on stages: Stage-2
+ Stage-0 depends on stages: Stage-3
+ Stage-4 depends on stages: Stage-0
+ Stage-1 depends on stages: Stage-3
+ Stage-5 depends on stages: Stage-1
STAGE PLANS:
- Stage: Stage-1
+ Stage: Stage-2
Tez
#### A masked pattern was here ####
Edges:
Reducer 2 <- Map 1 (SIMPLE_EDGE)
+ Reducer 3 <- Map 1 (SIMPLE_EDGE)
#### A masked pattern was here ####
Vertices:
Map 1
@@ -219,28 +278,55 @@ STAGE PLANS:
null sort order: z
sort order: +
Map-reduce partition columns: UDFToInteger(_col0) (type: int)
- Statistics: Num rows: 5 Data size: 1720 Basic stats: COMPLETE Column stats: PARTIAL
- value expressions: 'foo' (type: string), 'bar' (type: string), '2008-04-08' (type: string)
+ Statistics: Num rows: 5 Data size: 850 Basic stats: COMPLETE Column stats: PARTIAL
+ value expressions: '2008-04-08' (type: string)
+ Select Operator
+ expressions: 'foo' (type: string), 'bar' (type: string), '2008-04-08' (type: string)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 5 Data size: 1340 Basic stats: COMPLETE Column stats: PARTIAL
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ null sort order: a
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
+ Statistics: Num rows: 5 Data size: 1340 Basic stats: COMPLETE Column stats: PARTIAL
+ value expressions: _col1 (type: string), _col2 (type: string)
Execution mode: llap
LLAP IO: may be used (ACID table)
Reducer 2
Execution mode: llap
Reduce Operator Tree:
Select Operator
- expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string)
- outputColumnNames: _col0, _col1, _col2, _col3
- Statistics: Num rows: 5 Data size: 1720 Basic stats: COMPLETE Column stats: PARTIAL
+ expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), VALUE._col0 (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 5 Data size: 850 Basic stats: COMPLETE Column stats: PARTIAL
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 5 Data size: 850 Basic stats: COMPLETE Column stats: PARTIAL
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.acid_part
+ Write Type: DELETE
+ Reducer 3
+ Execution mode: llap
+ Reduce Operator Tree:
+ Select Operator
+ expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string), VALUE._col1 (type: string)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 5 Data size: 1340 Basic stats: COMPLETE Column stats: PARTIAL
File Output Operator
compressed: false
- Statistics: Num rows: 5 Data size: 1720 Basic stats: COMPLETE Column stats: PARTIAL
+ Statistics: Num rows: 5 Data size: 1340 Basic stats: COMPLETE Column stats: PARTIAL
table:
input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.acid_part
- Write Type: UPDATE
+ Write Type: INSERT
- Stage: Stage-2
+ Stage: Stage-3
Dependency Collection
Stage: Stage-0
@@ -254,9 +340,26 @@ STAGE PLANS:
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.acid_part
- Write Type: UPDATE
+ Write Type: DELETE
- Stage: Stage-3
+ Stage: Stage-4
+ Stats Work
+ Basic Stats Work:
+
+ Stage: Stage-1
+ Move Operator
+ tables:
+ partition:
+ ds
+ replace: false
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.acid_part
+ Write Type: INSERT
+
+ Stage: Stage-5
Stats Work
Basic Stats Work:
@@ -264,12 +367,17 @@ PREHOOK: query: update acid_part set value = 'bar' where key = 'foo' and ds in (
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_part
PREHOOK: Input: default@acid_part@ds=2008-04-08
+PREHOOK: Output: default@acid_part
PREHOOK: Output: default@acid_part@ds=2008-04-08
POSTHOOK: query: update acid_part set value = 'bar' where key = 'foo' and ds in ('2008-04-08')
POSTHOOK: type: QUERY
POSTHOOK: Input: default@acid_part
POSTHOOK: Input: default@acid_part@ds=2008-04-08
+POSTHOOK: Output: default@acid_part
+POSTHOOK: Output: default@acid_part@ds=2008-04-08
POSTHOOK: Output: default@acid_part@ds=2008-04-08
+POSTHOOK: Lineage: acid_part PARTITION(ds=2008-04-08).key SIMPLE []
+POSTHOOK: Lineage: acid_part PARTITION(ds=2008-04-08).value SIMPLE []
PREHOOK: query: select count(*) from acid_part where ds in ('2008-04-08')
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_part
@@ -381,24 +489,29 @@ PREHOOK: query: explain update acid_part_sdpo set value = 'bar' where key = 'foo
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_part_sdpo
PREHOOK: Input: default@acid_part_sdpo@ds=2008-04-08
+PREHOOK: Output: default@acid_part_sdpo
PREHOOK: Output: default@acid_part_sdpo@ds=2008-04-08
POSTHOOK: query: explain update acid_part_sdpo set value = 'bar' where key = 'foo' and ds='2008-04-08'
POSTHOOK: type: QUERY
POSTHOOK: Input: default@acid_part_sdpo
POSTHOOK: Input: default@acid_part_sdpo@ds=2008-04-08
+POSTHOOK: Output: default@acid_part_sdpo
POSTHOOK: Output: default@acid_part_sdpo@ds=2008-04-08
STAGE DEPENDENCIES:
- Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
- Stage-0 depends on stages: Stage-2
- Stage-3 depends on stages: Stage-0
+ Stage-2 is a root stage
+ Stage-3 depends on stages: Stage-2
+ Stage-0 depends on stages: Stage-3
+ Stage-4 depends on stages: Stage-0
+ Stage-1 depends on stages: Stage-3
+ Stage-5 depends on stages: Stage-1
STAGE PLANS:
- Stage: Stage-1
+ Stage: Stage-2
Tez
#### A masked pattern was here ####
Edges:
Reducer 2 <- Map 1 (SIMPLE_EDGE)
+ Reducer 3 <- Map 1 (SIMPLE_EDGE)
#### A masked pattern was here ####
Vertices:
Map 1
@@ -419,28 +532,55 @@ STAGE PLANS:
null sort order: z
sort order: +
Map-reduce partition columns: UDFToInteger(_col0) (type: int)
- Statistics: Num rows: 5 Data size: 1720 Basic stats: COMPLETE Column stats: PARTIAL
- value expressions: 'foo' (type: string), 'bar' (type: string), '2008-04-08' (type: string)
+ Statistics: Num rows: 5 Data size: 850 Basic stats: COMPLETE Column stats: PARTIAL
+ value expressions: '2008-04-08' (type: string)
+ Select Operator
+ expressions: 'foo' (type: string), 'bar' (type: string), '2008-04-08' (type: string)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 5 Data size: 1340 Basic stats: COMPLETE Column stats: PARTIAL
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ null sort order: a
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
+ Statistics: Num rows: 5 Data size: 1340 Basic stats: COMPLETE Column stats: PARTIAL
+ value expressions: _col1 (type: string), _col2 (type: string)
Execution mode: llap
LLAP IO: may be used (ACID table)
Reducer 2
Execution mode: llap
Reduce Operator Tree:
Select Operator
- expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string)
- outputColumnNames: _col0, _col1, _col2, _col3
- Statistics: Num rows: 5 Data size: 1720 Basic stats: COMPLETE Column stats: PARTIAL
+ expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), VALUE._col0 (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 5 Data size: 850 Basic stats: COMPLETE Column stats: PARTIAL
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 5 Data size: 850 Basic stats: COMPLETE Column stats: PARTIAL
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.acid_part_sdpo
+ Write Type: DELETE
+ Reducer 3
+ Execution mode: llap
+ Reduce Operator Tree:
+ Select Operator
+ expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string), VALUE._col1 (type: string)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 5 Data size: 1340 Basic stats: COMPLETE Column stats: PARTIAL
File Output Operator
compressed: false
- Statistics: Num rows: 5 Data size: 1720 Basic stats: COMPLETE Column stats: PARTIAL
+ Statistics: Num rows: 5 Data size: 1340 Basic stats: COMPLETE Column stats: PARTIAL
table:
input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.acid_part_sdpo
- Write Type: UPDATE
+ Write Type: INSERT
- Stage: Stage-2
+ Stage: Stage-3
Dependency Collection
Stage: Stage-0
@@ -454,9 +594,26 @@ STAGE PLANS:
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.acid_part_sdpo
- Write Type: UPDATE
+ Write Type: DELETE
- Stage: Stage-3
+ Stage: Stage-4
+ Stats Work
+ Basic Stats Work:
+
+ Stage: Stage-1
+ Move Operator
+ tables:
+ partition:
+ ds
+ replace: false
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.acid_part_sdpo
+ Write Type: INSERT
+
+ Stage: Stage-5
Stats Work
Basic Stats Work:
@@ -464,12 +621,17 @@ PREHOOK: query: update acid_part_sdpo set value = 'bar' where key = 'foo' and ds
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_part_sdpo
PREHOOK: Input: default@acid_part_sdpo@ds=2008-04-08
+PREHOOK: Output: default@acid_part_sdpo
PREHOOK: Output: default@acid_part_sdpo@ds=2008-04-08
POSTHOOK: query: update acid_part_sdpo set value = 'bar' where key = 'foo' and ds='2008-04-08'
POSTHOOK: type: QUERY
POSTHOOK: Input: default@acid_part_sdpo
POSTHOOK: Input: default@acid_part_sdpo@ds=2008-04-08
+POSTHOOK: Output: default@acid_part_sdpo
+POSTHOOK: Output: default@acid_part_sdpo@ds=2008-04-08
POSTHOOK: Output: default@acid_part_sdpo@ds=2008-04-08
+POSTHOOK: Lineage: acid_part_sdpo PARTITION(ds=2008-04-08).key SIMPLE []
+POSTHOOK: Lineage: acid_part_sdpo PARTITION(ds=2008-04-08).value SIMPLE []
PREHOOK: query: select count(*) from acid_part_sdpo where ds='2008-04-08'
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_part_sdpo
@@ -485,24 +647,29 @@ PREHOOK: query: explain update acid_part_sdpo set value = 'bar' where key = 'foo
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_part_sdpo
PREHOOK: Input: default@acid_part_sdpo@ds=2008-04-08
+PREHOOK: Output: default@acid_part_sdpo
PREHOOK: Output: default@acid_part_sdpo@ds=2008-04-08
POSTHOOK: query: explain update acid_part_sdpo set value = 'bar' where key = 'foo' and ds in ('2008-04-08')
POSTHOOK: type: QUERY
POSTHOOK: Input: default@acid_part_sdpo
POSTHOOK: Input: default@acid_part_sdpo@ds=2008-04-08
+POSTHOOK: Output: default@acid_part_sdpo
POSTHOOK: Output: default@acid_part_sdpo@ds=2008-04-08
STAGE DEPENDENCIES:
- Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
- Stage-0 depends on stages: Stage-2
- Stage-3 depends on stages: Stage-0
+ Stage-2 is a root stage
+ Stage-3 depends on stages: Stage-2
+ Stage-0 depends on stages: Stage-3
+ Stage-4 depends on stages: Stage-0
+ Stage-1 depends on stages: Stage-3
+ Stage-5 depends on stages: Stage-1
STAGE PLANS:
- Stage: Stage-1
+ Stage: Stage-2
Tez
#### A masked pattern was here ####
Edges:
Reducer 2 <- Map 1 (SIMPLE_EDGE)
+ Reducer 3 <- Map 1 (SIMPLE_EDGE)
#### A masked pattern was here ####
Vertices:
Map 1
@@ -523,28 +690,55 @@ STAGE PLANS:
null sort order: z
sort order: +
Map-reduce partition columns: UDFToInteger(_col0) (type: int)
- Statistics: Num rows: 5 Data size: 1720 Basic stats: COMPLETE Column stats: PARTIAL
- value expressions: 'foo' (type: string), 'bar' (type: string), '2008-04-08' (type: string)
+ Statistics: Num rows: 5 Data size: 850 Basic stats: COMPLETE Column stats: PARTIAL
+ value expressions: '2008-04-08' (type: string)
+ Select Operator
+ expressions: 'foo' (type: string), 'bar' (type: string), '2008-04-08' (type: string)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 5 Data size: 1340 Basic stats: COMPLETE Column stats: PARTIAL
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ null sort order: a
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
+ Statistics: Num rows: 5 Data size: 1340 Basic stats: COMPLETE Column stats: PARTIAL
+ value expressions: _col1 (type: string), _col2 (type: string)
Execution mode: llap
LLAP IO: may be used (ACID table)
Reducer 2
Execution mode: llap
Reduce Operator Tree:
Select Operator
- expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string)
- outputColumnNames: _col0, _col1, _col2, _col3
- Statistics: Num rows: 5 Data size: 1720 Basic stats: COMPLETE Column stats: PARTIAL
+ expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), VALUE._col0 (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 5 Data size: 850 Basic stats: COMPLETE Column stats: PARTIAL
File Output Operator
compressed: false
- Statistics: Num rows: 5 Data size: 1720 Basic stats: COMPLETE Column stats: PARTIAL
+ Statistics: Num rows: 5 Data size: 850 Basic stats: COMPLETE Column stats: PARTIAL
table:
input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.acid_part_sdpo
- Write Type: UPDATE
+ Write Type: DELETE
+ Reducer 3
+ Execution mode: llap
+ Reduce Operator Tree:
+ Select Operator
+ expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string), VALUE._col1 (type: string)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 5 Data size: 1340 Basic stats: COMPLETE Column stats: PARTIAL
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 5 Data size: 1340 Basic stats: COMPLETE Column stats: PARTIAL
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.acid_part_sdpo
+ Write Type: INSERT
- Stage: Stage-2
+ Stage: Stage-3
Dependency Collection
Stage: Stage-0
@@ -558,9 +752,26 @@ STAGE PLANS:
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.acid_part_sdpo
- Write Type: UPDATE
+ Write Type: DELETE
- Stage: Stage-3
+ Stage: Stage-4
+ Stats Work
+ Basic Stats Work:
+
+ Stage: Stage-1
+ Move Operator
+ tables:
+ partition:
+ ds
+ replace: false
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.acid_part_sdpo
+ Write Type: INSERT
+
+ Stage: Stage-5
Stats Work
Basic Stats Work:
@@ -568,12 +779,17 @@ PREHOOK: query: update acid_part_sdpo set value = 'bar' where key = 'foo' and ds
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_part_sdpo
PREHOOK: Input: default@acid_part_sdpo@ds=2008-04-08
+PREHOOK: Output: default@acid_part_sdpo
PREHOOK: Output: default@acid_part_sdpo@ds=2008-04-08
POSTHOOK: query: update acid_part_sdpo set value = 'bar' where key = 'foo' and ds in ('2008-04-08')
POSTHOOK: type: QUERY
POSTHOOK: Input: default@acid_part_sdpo
POSTHOOK: Input: default@acid_part_sdpo@ds=2008-04-08
+POSTHOOK: Output: default@acid_part_sdpo
POSTHOOK: Output: default@acid_part_sdpo@ds=2008-04-08
+POSTHOOK: Output: default@acid_part_sdpo@ds=2008-04-08
+POSTHOOK: Lineage: acid_part_sdpo PARTITION(ds=2008-04-08).key SIMPLE []
+POSTHOOK: Lineage: acid_part_sdpo PARTITION(ds=2008-04-08).value SIMPLE []
PREHOOK: query: select count(*) from acid_part_sdpo where ds in ('2008-04-08')
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_part_sdpo
@@ -694,24 +910,29 @@ PREHOOK: query: explain update acid_2L_part set value = 'bar' where key = 'foo'
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_2l_part
PREHOOK: Input: default@acid_2l_part@ds=2008-04-08/hr=11
+PREHOOK: Output: default@acid_2l_part
PREHOOK: Output: default@acid_2l_part@ds=2008-04-08/hr=11
POSTHOOK: query: explain update acid_2L_part set value = 'bar' where key = 'foo' and ds='2008-04-08' and hr=11
POSTHOOK: type: QUERY
POSTHOOK: Input: default@acid_2l_part
POSTHOOK: Input: default@acid_2l_part@ds=2008-04-08/hr=11
+POSTHOOK: Output: default@acid_2l_part
POSTHOOK: Output: default@acid_2l_part@ds=2008-04-08/hr=11
STAGE DEPENDENCIES:
- Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
- Stage-0 depends on stages: Stage-2
- Stage-3 depends on stages: Stage-0
+ Stage-2 is a root stage
+ Stage-3 depends on stages: Stage-2
+ Stage-0 depends on stages: Stage-3
+ Stage-4 depends on stages: Stage-0
+ Stage-1 depends on stages: Stage-3
+ Stage-5 depends on stages: Stage-1
STAGE PLANS:
- Stage: Stage-1
+ Stage: Stage-2
Tez
#### A masked pattern was here ####
Edges:
Reducer 2 <- Map 1 (SIMPLE_EDGE)
+ Reducer 3 <- Map 1 (SIMPLE_EDGE)
#### A masked pattern was here ####
Vertices:
Map 1
@@ -732,28 +953,55 @@ STAGE PLANS:
null sort order: z
sort order: +
Map-reduce partition columns: UDFToInteger(_col0) (type: int)
- Statistics: Num rows: 5 Data size: 1740 Basic stats: COMPLETE Column stats: PARTIAL
- value expressions: 'foo' (type: string), 'bar' (type: string), '2008-04-08' (type: string), 11 (type: int)
+ Statistics: Num rows: 5 Data size: 870 Basic stats: COMPLETE Column stats: PARTIAL
+ value expressions: '2008-04-08' (type: string), 11 (type: int)
+ Select Operator
+ expressions: 'foo' (type: string), 'bar' (type: string), '2008-04-08' (type: string), 11 (type: int)
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Statistics: Num rows: 5 Data size: 1360 Basic stats: COMPLETE Column stats: PARTIAL
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ null sort order: a
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
+ Statistics: Num rows: 5 Data size: 1360 Basic stats: COMPLETE Column stats: PARTIAL
+ value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: int)
Execution mode: llap
LLAP IO: may be used (ACID table)
Reducer 2
Execution mode: llap
Reduce Operator Tree:
Select Operator
- expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int)
- outputColumnNames: _col0, _col1, _col2, _col3, _col4
- Statistics: Num rows: 5 Data size: 1740 Basic stats: COMPLETE Column stats: PARTIAL
+ expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), VALUE._col0 (type: string), VALUE._col1 (type: int)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 5 Data size: 870 Basic stats: COMPLETE Column stats: PARTIAL
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 5 Data size: 870 Basic stats: COMPLETE Column stats: PARTIAL
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.acid_2l_part
+ Write Type: DELETE
+ Reducer 3
+ Execution mode: llap
+ Reduce Operator Tree:
+ Select Operator
+ expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: int)
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Statistics: Num rows: 5 Data size: 1360 Basic stats: COMPLETE Column stats: PARTIAL
File Output Operator
compressed: false
- Statistics: Num rows: 5 Data size: 1740 Basic stats: COMPLETE Column stats: PARTIAL
+ Statistics: Num rows: 5 Data size: 1360 Basic stats: COMPLETE Column stats: PARTIAL
table:
input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.acid_2l_part
- Write Type: UPDATE
+ Write Type: INSERT
- Stage: Stage-2
+ Stage: Stage-3
Dependency Collection
Stage: Stage-0
@@ -768,9 +1016,27 @@ STAGE PLANS:
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.acid_2l_part
- Write Type: UPDATE
+ Write Type: DELETE
- Stage: Stage-3
+ Stage: Stage-4
+ Stats Work
+ Basic Stats Work:
+
+ Stage: Stage-1
+ Move Operator
+ tables:
+ partition:
+ ds
+ hr
+ replace: false
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.acid_2l_part
+ Write Type: INSERT
+
+ Stage: Stage-5
Stats Work
Basic Stats Work:
@@ -778,12 +1044,17 @@ PREHOOK: query: update acid_2L_part set value = 'bar' where key = 'foo' and ds='
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_2l_part
PREHOOK: Input: default@acid_2l_part@ds=2008-04-08/hr=11
+PREHOOK: Output: default@acid_2l_part
PREHOOK: Output: default@acid_2l_part@ds=2008-04-08/hr=11
POSTHOOK: query: update acid_2L_part set value = 'bar' where key = 'foo' and ds='2008-04-08' and hr=11
POSTHOOK: type: QUERY
POSTHOOK: Input: default@acid_2l_part
POSTHOOK: Input: default@acid_2l_part@ds=2008-04-08/hr=11
+POSTHOOK: Output: default@acid_2l_part
+POSTHOOK: Output: default@acid_2l_part@ds=2008-04-08/hr=11
POSTHOOK: Output: default@acid_2l_part@ds=2008-04-08/hr=11
+POSTHOOK: Lineage: acid_2l_part PARTITION(ds=2008-04-08,hr=11).key SIMPLE []
+POSTHOOK: Lineage: acid_2l_part PARTITION(ds=2008-04-08,hr=11).value SIMPLE []
PREHOOK: query: select count(*) from acid_2L_part where ds='2008-04-08' and hr=11
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_2l_part
@@ -800,6 +1071,7 @@ PREHOOK: type: QUERY
PREHOOK: Input: default@acid_2l_part
PREHOOK: Input: default@acid_2l_part@ds=2008-04-08/hr=11
PREHOOK: Input: default@acid_2l_part@ds=2008-04-08/hr=12
+PREHOOK: Output: default@acid_2l_part
PREHOOK: Output: default@acid_2l_part@ds=2008-04-08/hr=11
PREHOOK: Output: default@acid_2l_part@ds=2008-04-08/hr=12
POSTHOOK: query: explain update acid_2L_part set value = 'bar' where key = 'foo' and ds='2008-04-08' and hr>=11
@@ -807,20 +1079,24 @@ POSTHOOK: type: QUERY
POSTHOOK: Input: default@acid_2l_part
POSTHOOK: Input: default@acid_2l_part@ds=2008-04-08/hr=11
POSTHOOK: Input: default@acid_2l_part@ds=2008-04-08/hr=12
+POSTHOOK: Output: default@acid_2l_part
POSTHOOK: Output: default@acid_2l_part@ds=2008-04-08/hr=11
POSTHOOK: Output: default@acid_2l_part@ds=2008-04-08/hr=12
STAGE DEPENDENCIES:
- Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
- Stage-0 depends on stages: Stage-2
- Stage-3 depends on stages: Stage-0
+ Stage-2 is a root stage
+ Stage-3 depends on stages: Stage-2
+ Stage-0 depends on stages: Stage-3
+ Stage-4 depends on stages: Stage-0
+ Stage-1 depends on stages: Stage-3
+ Stage-5 depends on stages: Stage-1
STAGE PLANS:
- Stage: Stage-1
+ Stage: Stage-2
Tez
#### A masked pattern was here ####
Edges:
Reducer 2 <- Map 1 (SIMPLE_EDGE)
+ Reducer 3 <- Map 1 (SIMPLE_EDGE)
#### A masked pattern was here ####
Vertices:
Map 1
@@ -836,33 +1112,64 @@ STAGE PLANS:
expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), hr (type: int)
outputColumnNames: _col0, _col4
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: 3480 Basic stats: COMPLETE Column stats: PARTIAL
- value expressions: 'foo' (type: string), 'bar' (type: string), '2008-04-08' (type: string), _col4 (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)
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Statistics: Num rows: 10 Data size: 2720 Basic stats: COMPLETE Column stats: PARTIAL
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ null sort order: a
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
+ Statistics: Num rows: 10 Data size: 2720 Basic stats: COMPLETE Column stats: PARTIAL
+ value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: int)
Execution mode: llap
LLAP IO: may be used (ACID table)
Reducer 2
Execution mode: llap
Reduce Operator Tree:
Select Operator
- expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int)
- outputColumnNames: _col0, _col1, _col2, _col3, _col4
- Statistics: Num rows: 10 Data size: 3480 Basic stats: COMPLETE Column stats: PARTIAL
+ expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), VALUE._col0 (type: string), VALUE._col1 (type: int)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 10 Data size: 1740 Basic stats: COMPLETE Column stats: PARTIAL
File Output Operator
compressed: false
- Statistics: Num rows: 10 Data size: 3480 Basic stats: COMPLETE Column stats: PARTIAL
+ Statistics: Num rows: 10 Data size: 1740 Basic stats: COMPLETE Column stats: PARTIAL
table:
input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.acid_2l_part
- Write Type: UPDATE
+ Write Type: DELETE
+ Reducer 3
+ Execution mode: llap
+ Reduce Operator Tree:
+ Select Operator
+ expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: int)
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Statistics: Num rows: 10 Data size: 2720 Basic stats: COMPLETE Column stats: PARTIAL
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 10 Data size: 2720 Basic stats: COMPLETE Column stats: PARTIAL
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.acid_2l_part
+ Write Type: INSERT
- Stage: Stage-2
+ Stage: Stage-3
Dependency Collection
Stage: Stage-0
@@ -877,9 +1184,27 @@ STAGE PLANS:
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.acid_2l_part
- Write Type: UPDATE
+ Write Type: DELETE
- Stage: Stage-3
+ Stage: Stage-4
+ Stats Work
+ Basic Stats Work:
+
+ Stage: Stage-1
+ Move Operator
+ tables:
+ partition:
+ ds
+ hr
+ replace: false
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.acid_2l_part
+ Write Type: INSERT
+
+ Stage: Stage-5
Stats Work
Basic Stats Work:
@@ -888,6 +1213,7 @@ PREHOOK: type: QUERY
PREHOOK: Input: default@acid_2l_part
PREHOOK: Input: default@acid_2l_part@ds=2008-04-08/hr=11
PREHOOK: Input: default@acid_2l_part@ds=2008-04-08/hr=12
+PREHOOK: Output: default@acid_2l_part
PREHOOK: Output: default@acid_2l_part@ds=2008-04-08/hr=11
PREHOOK: Output: default@acid_2l_part@ds=2008-04-08/hr=12
POSTHOOK: query: update acid_2L_part set value = 'bar' where key = 'foo' and ds='2008-04-08' and hr>=11
@@ -895,8 +1221,12 @@ POSTHOOK: type: QUERY
POSTHOOK: Input: default@acid_2l_part
POSTHOOK: Input: default@acid_2l_part@ds=2008-04-08/hr=11
POSTHOOK: Input: default@acid_2l_part@ds=2008-04-08/hr=12
+POSTHOOK: Output: default@acid_2l_part
+POSTHOOK: Output: default@acid_2l_part@ds=2008-04-08/hr=11
POSTHOOK: Output: default@acid_2l_part@ds=2008-04-08/hr=11
POSTHOOK: Output: default@acid_2l_part@ds=2008-04-08/hr=12
+POSTHOOK: Lineage: acid_2l_part PARTITION(ds=2008-04-08,hr=11).key SIMPLE []
+POSTHOOK: Lineage: acid_2l_part PARTITION(ds=2008-04-08,hr=11).value SIMPLE []
PREHOOK: query: select count(*) from acid_2L_part where ds='2008-04-08' and hr>=11
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_2l_part
@@ -1156,24 +1486,29 @@ PREHOOK: query: explain update acid_2L_part_sdpo set value = 'bar' where key = '
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_2l_part_sdpo
PREHOOK: Input: default@acid_2l_part_sdpo@ds=2008-04-08/hr=11
+PREHOOK: Output: default@acid_2l_part_sdpo
PREHOOK: Output: default@acid_2l_part_sdpo@ds=2008-04-08/hr=11
POSTHOOK: query: explain update acid_2L_part_sdpo set value = 'bar' where key = 'foo' and ds='2008-04-08' and hr=11
POSTHOOK: type: QUERY
POSTHOOK: Input: default@acid_2l_part_sdpo
POSTHOOK: Input: default@acid_2l_part_sdpo@ds=2008-04-08/hr=11
+POSTHOOK: Output: default@acid_2l_part_sdpo
POSTHOOK: Output: default@acid_2l_part_sdpo@ds=2008-04-08/hr=11
STAGE DEPENDENCIES:
- Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
- Stage-0 depends on stages: Stage-2
- Stage-3 depends on stages: Stage-0
+ Stage-2 is a root stage
+ Stage-3 depends on stages: Stage-2
+ Stage-0 depends on stages: Stage-3
+ Stage-4 depends on stages: Stage-0
+ Stage-1 depends on stages: Stage-3
+ Stage-5 depends on stages: Stage-1
STAGE PLANS:
- Stage: Stage-1
+ Stage: Stage-2
Tez
#### A masked pattern was here ####
Edges:
Reducer 2 <- Map 1 (SIMPLE_EDGE)
+ Reducer 3 <- Map 1 (SIMPLE_EDGE)
#### A masked pattern was here ####
Vertices:
Map 1
@@ -1194,28 +1529,55 @@ STAGE PLANS:
null sort order: z
sort order: +
Map-reduce partition columns: UDFToInteger(_col0) (type: int)
- Statistics: Num rows: 5 Data size: 1740 Basic stats: COMPLETE Column stats: PARTIAL
- value expressions: 'foo' (type: string), 'bar' (type: string), '2008-04-08' (type: string), 11 (type: int)
+ Statistics: Num rows: 5 Data size: 870 Basic stats: COMPLETE Column stats: PARTIAL
+ value expressions: '2008-04-08' (type: string), 11 (type: int)
+ Select Operator
+ expressions: 'foo' (type: string), 'bar' (type: string), '2008-04-08' (type: string), 11 (type: int)
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Statistics: Num rows: 5 Data size: 1360 Basic stats: COMPLETE Column stats: PARTIAL
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ null sort order: a
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
+ Statistics: Num rows: 5 Data size: 1360 Basic stats: COMPLETE Column stats: PARTIAL
+ value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: int)
Execution mode: llap
LLAP IO: may be used (ACID table)
Reducer 2
Execution mode: llap
Reduce Operator Tree:
Select Operator
- expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int)
- outputColumnNames: _col0, _col1, _col2, _col3, _col4
- Statistics: Num rows: 5 Data size: 1740 Basic stats: COMPLETE Column stats: PARTIAL
+ expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), VALUE._col0 (type: string), VALUE._col1 (type: int)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 5 Data size: 870 Basic stats: COMPLETE Column stats: PARTIAL
File Output Operator
compressed: false
- Statistics: Num rows: 5 Data size: 1740 Basic stats: COMPLETE Column stats: PARTIAL
+ Statistics: Num rows: 5 Data size: 870 Basic stats: COMPLETE Column stats: PARTIAL
table:
input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.acid_2l_part_sdpo
- Write Type: UPDATE
+ Write Type: DELETE
+ Reducer 3
+ Execution mode: llap
+ Reduce Operator Tree:
+ Select Operator
+ expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: int)
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Statistics: Num rows: 5 Data size: 1360 Basic stats: COMPLETE Column stats: PARTIAL
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 5 Data size: 1360 Basic stats: COMPLETE Column stats: PARTIAL
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.acid_2l_part_sdpo
+ Write Type: INSERT
- Stage: Stage-2
+ Stage: Stage-3
Dependency Collection
Stage: Stage-0
@@ -1230,9 +1592,27 @@ STAGE PLANS:
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.acid_2l_part_sdpo
- Write Type: UPDATE
+ Write Type: DELETE
- Stage: Stage-3
+ Stage: Stage-4
+ Stats Work
+ Basic Stats Work:
+
+ Stage: Stage-1
+ Move Operator
+ tables:
+ partition:
+ ds
+ hr
+ replace: false
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.acid_2l_part_sdpo
+ Write Type: INSERT
+
+ Stage: Stage-5
Stats Work
Basic Stats Work:
@@ -1240,12 +1620,17 @@ PREHOOK: query: update acid_2L_part_sdpo set value = 'bar' where key = 'foo' and
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_2l_part_sdpo
PREHOOK: Input: default@acid_2l_part_sdpo@ds=2008-04-08/hr=11
+PREHOOK: Output: default@acid_2l_part_sdpo
PREHOOK: Output: default@acid_2l_part_sdpo@ds=2008-04-08/hr=11
POSTHOOK: query: update acid_2L_part_sdpo set value = 'bar' where key = 'foo' and ds='2008-04-08' and hr=11
POSTHOOK: type: QUERY
POSTHOOK: Input: default@acid_2l_part_sdpo
POSTHOOK: Input: default@acid_2l_part_sdpo@ds=2008-04-08/hr=11
+POSTHOOK: Output: default@acid_2l_part_sdpo
+POSTHOOK: Output: default@acid_2l_part_sdpo@ds=2008-04-08/hr=11
POSTHOOK: Output: default@acid_2l_part_sdpo@ds=2008-04-08/hr=11
+POSTHOOK: Lineage: acid_2l_part_sdpo PARTITION(ds=2008-04-08,hr=11).key SIMPLE []
+POSTHOOK: Lineage: acid_2l_part_sdpo PARTITION(ds=2008-04-08,hr=11).value SIMPLE []
PREHOOK: query: select count(*) from acid_2L_part_sdpo where ds='2008-04-08' and hr=11
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_2l_part_sdpo
@@ -1262,6 +1647,7 @@ PREHOOK: type: QUERY
PREHOOK: Input: default@acid_2l_part_sdpo
PREHOOK: Input: default@acid_2l_part_sdpo@ds=2008-04-08/hr=11
PREHOOK: Input: default@acid_2l_part_sdpo@ds=2008-04-08/hr=12
+PREHOOK: Output: default@acid_2l_part_sdpo
PREHOOK: Output: default@acid_2l_part_sdpo@ds=2008-04-08/hr=11
PREHOOK: Output: default@acid_2l_part_sdpo@ds=2008-04-08/hr=12
POSTHOOK: query: explain update acid_2L_part_sdpo set value = 'bar' where key = 'foo' and ds='2008-04-08' and hr>=11
@@ -1269,20 +1655,24 @@ POSTHOOK: type: QUERY
POSTHOOK: Input: default@acid_2l_part_sdpo
POSTHOOK: Input: default@acid_2l_part_sdpo@ds=2008-04-08/hr=11
POSTHOOK: Input: default@acid_2l_part_sdpo@ds=2008-04-08/hr=12
+POSTHOOK: Output: default@acid_2l_part_sdpo
POSTHOOK: Output: default@acid_2l_part_sdpo@ds=2008-04-08/hr=11
POSTHOOK: Output: default@acid_2l_part_sdpo@ds=2008-04-08/hr=12
STAGE DEPENDENCIES:
- Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
- Stage-0 depends on stages: Stage-2
- Stage-3 depends on stages: Stage-0
+ Stage-2 is a root stage
+ Stage-3 depends on stages: Stage-2
+ Stage-0 depends on stages: Stage-3
+ Stage-4 depends on stages: Stage-0
+ Stage-1 depends on stages: Stage-3
+ Stage-5 depends on stages: Stage-1
STAGE PLANS:
- Stage: Stage-1
+ Stage: Stage-2
Tez
#### A masked pattern was here ####
Edges:
Reducer 2 <- Map 1 (SIMPLE_EDGE)
+ Reducer 3 <- Map 1 (SIMPLE_EDGE)
#### A masked pattern was here ####
Vertices:
Map 1
@@ -1298,33 +1688,63 @@ STAGE PLANS:
expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), hr (type: int)
outputColumnNames: _col0, _col4
Statistics: Num rows: 10 Data size: 800 Basic stats: COMPLETE Column stats: PARTIAL
- Reduce Output Operator
- key expressions: '2008-04-08' (type: string), _col4 (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), _col4 (type: int)
- Statistics: Num rows: 10 Data size: 3480 Basic stats: COMPLETE Column stats: PARTIAL
- value expressions: 'foo' (type: string), 'bar' (type: string)
+ 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)
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Statistics: Num rows: 10 Data size: 2720 Basic stats: COMPLETE Column stats: PARTIAL
+ Reduce Output Operator
+ key expressions: _col2 (type: string), _col3 (type: int), _bucket_number (type: string), _col0 (type: string)
+ null sort order: aaaa
+ sort order: ++++
+ Map-reduce partition columns: _col2 (type: string), _col3 (type: int)
+ Statistics: Num rows: 10 Data size: 2720 Basic stats: COMPLETE Column stats: PARTIAL
+ value expressions: _col1 (type: string)
Execution mode: llap
LLAP IO: may be used (ACID table)
Reducer 2
Execution mode: llap
Reduce Operator Tree:
Select Operator
- expressions: KEY._col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), VALUE._col1 (type: string), VALUE._col2 (type: string), KEY._col3 (type: string), KEY._col4 (type: int), KEY._bucket_number (type: string)
- outputColumnNames: _col0, _col1, _col2, _col3, _col4, _bucket_number
+ expressions: KEY._col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), KEY._col1 (type: string), KEY._col2 (type: int), KEY._bucket_number (type: string)
+ outputColumnNames: _col0, _col1, _col2, _bucket_number
+ File Output Operator
+ compressed: false
+ Dp Sort State: PARTITION_BUCKET_SORTED
+ Statistics: Num rows: 10 Data size: 1740 Basic stats: COMPLETE Column stats: PARTIAL
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.acid_2l_part_sdpo
+ Write Type: DELETE
+ Reducer 3
+ Execution mode: llap
+ Reduce Operator Tree:
+ Select Operator
+ expressions: KEY._col0 (type: string), VALUE._col1 (type: string), KEY._col2 (type: string), KEY._col3 (type: int), KEY._bucket_number (type: string)
+ outputColumnNames: _col0, _col1, _col2, _col3, _bucket_number
File Output Operator
compressed: false
Dp Sort State: PARTITION_BUCKET_SORTED
- Statistics: Num rows: 10 Data size: 3480 Basic stats: COMPLETE Column stats: PARTIAL
+ Statistics: Num rows: 10 Data size: 2720 Basic stats: COMPLETE Column stats: PARTIAL
table:
input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.acid_2l_part_sdpo
- Write Type: UPDATE
+ Write Type: INSERT
- Stage: Stage-2
+ Stage: Stage-3
Dependency Collection
Stage: Stage-0
@@ -1339,9 +1759,27 @@ STAGE PLANS:
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.acid_2l_part_sdpo
- Write Type: UPDATE
+ Write Type: DELETE
+
+ Stage: Stage-4
+ Stats Work
+ Basic Stats Work:
+
+ Stage: Stage-1
+ Move Operator
+ tables:
+ partition:
+ ds
+ hr
+ replace: false
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.acid_2l_part_sdpo
+ Write Type: INSERT
- Stage: Stage-3
+ Stage: Stage-5
Stats Work
Basic Stats Work:
@@ -1350,6 +1788,7 @@ PREHOOK: type: QUERY
PREHOOK: Input: default@acid_2l_part_sdpo
PREHOOK: Input: default@acid_2l_part_sdpo@ds=2008-04-08/hr=11
PREHOOK: Input: default@acid_2l_part_sdpo@ds=2008-04-08/hr=12
+PREHOOK: Output: default@acid_2l_part_sdpo
PREHOOK: Output: default@acid_2l_part_sdpo@ds=2008-04-08/hr=11
PREHOOK: Output: default@acid_2l_part_sdpo@ds=2008-04-08/hr=12
POSTHOOK: query: update acid_2L_part_sdpo set value = 'bar' where key = 'foo' and ds='2008-04-08' and hr>=11
@@ -1357,8 +1796,12 @@ POSTHOOK: type: QUERY
POSTHOOK: Input: default@acid_2l_part_sdpo
POSTHOOK: Input: default@acid_2l_part_sdpo@ds=2008-04-08/hr=11
POSTHOOK: Input: default@acid_2l_part_sdpo@ds=2008-04-08/hr=12
+POSTHOOK: Output: default@acid_2l_part_sdpo
+POSTHOOK: Output: default@acid_2l_part_sdpo@ds=2008-04-08/hr=11
POSTHOOK: Output: default@acid_2l_part_sdpo@ds=2008-04-08/hr=11
POSTHOOK: Output: default@acid_2l_part_sdpo@ds=2008-04-08/hr=12
+POSTHOOK: Lineage: acid_2l_part_sdpo PARTITION(ds=2008-04-08,hr=11).key SIMPLE []
+POSTHOOK: Lineage: acid_2l_part_sdpo PARTITION(ds=2008-04-08,hr=11).value SIMPLE []
PREHOOK: query: select count(*) from acid_2L_part_sdpo where ds='2008-04-08' and hr>=11
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_2l_part_sdpo
@@ -1617,24 +2060,29 @@ PREHOOK: query: explain update acid_2L_part_sdpo_no_cp set value = 'bar' where k
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_2l_part_sdpo_no_cp
PREHOOK: Input: default@acid_2l_part_sdpo_no_cp@ds=2008-04-08/hr=11
+PREHOOK: Output: default@acid_2l_part_sdpo_no_cp
PREHOOK: Output: default@acid_2l_part_sdpo_no_cp@ds=2008-04-08/hr=11
POSTHOOK: query: explain update acid_2L_part_sdpo_no_cp set value = 'bar' where key = 'foo' and ds='2008-04-08' and hr=11
POSTHOOK: type: QUERY
POSTHOOK: Input: default@acid_2l_part_sdpo_no_cp
POSTHOOK: Input: default@acid_2l_part_sdpo_no_cp@ds=2008-04-08/hr=11
+POSTHOOK: Output: default@acid_2l_part_sdpo_no_cp
POSTHOOK: Output: default@acid_2l_part_sdpo_no_cp@ds=2008-04-08/hr=11
STAGE DEPENDENCIES:
- Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
- Stage-0 depends on stages: Stage-2
- Stage-3 depends on stages: Stage-0
+ Stage-2 is a root stage
+ Stage-3 depends on stages: Stage-2
+ Stage-0 depends on stages: Stage-3
+ Stage-4 depends on stages: Stage-0
+ Stage-1 depends on stages: Stage-3
+ Stage-5 depends on stages: Stage-1
STAGE PLANS:
- Stage: Stage-1
+ Stage: Stage-2
Tez
#### A masked pattern was here ####
Edges:
Reducer 2 <- Map 1 (SIMPLE_EDGE)
+ Reducer 3 <- Map 1 (SIMPLE_EDGE)
#### A masked pattern was here ####
Vertices:
Map 1
@@ -1655,28 +2103,55 @@ STAGE PLANS:
null sort order: z
sort order: +
Map-reduce partition columns: UDFToInteger(_col0) (type: int)
- Statistics: Num rows: 5 Data size: 1740 Basic stats: COMPLETE Column stats: PARTIAL
- value expressions: 'foo' (type: string), 'bar' (type: string), '2008-04-08' (type: string), 11 (type: int)
+ Statistics: Num rows: 5 Data size: 870 Basic stats: COMPLETE Column stats: PARTIAL
+ value expressions: '2008-04-08' (type: string), 11 (type: int)
+ Select Operator
+ expressions: 'foo' (type: string), 'bar' (type: string), '2008-04-08' (type: string), 11 (type: int)
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Statistics: Num rows: 5 Data size: 1360 Basic stats: COMPLETE Column stats: PARTIAL
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ null sort order: a
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
+ Statistics: Num rows: 5 Data size: 1360 Basic stats: COMPLETE Column stats: PARTIAL
+ value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: int)
Execution mode: llap
LLAP IO: may be used (ACID table)
Reducer 2
Execution mode: llap
Reduce Operator Tree:
Select Operator
- expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int)
- outputColumnNames: _col0, _col1, _col2, _col3, _col4
- Statistics: Num rows: 5 Data size: 1740 Basic stats: COMPLETE Column stats: PARTIAL
+ expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), VALUE._col0 (type: string), VALUE._col1 (type: int)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 5 Data size: 870 Basic stats: COMPLETE Column stats: PARTIAL
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 5 Data size: 870 Basic stats: COMPLETE Column stats: PARTIAL
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.acid_2l_part_sdpo_no_cp
+ Write Type: DELETE
+ Reducer 3
+ Execution mode: llap
+ Reduce Operator Tree:
+ Select Operator
+ expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: int)
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Statistics: Num rows: 5 Data size: 1360 Basic stats: COMPLETE Column stats: PARTIAL
File Output Operator
compressed: false
- Statistics: Num rows: 5 Data size: 1740 Basic stats: COMPLETE Column stats: PARTIAL
+ Statistics: Num rows: 5 Data size: 1360 Basic stats: COMPLETE Column stats: PARTIAL
table:
input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.acid_2l_part_sdpo_no_cp
- Write Type: UPDATE
+ Write Type: INSERT
- Stage: Stage-2
+ Stage: Stage-3
Dependency Collection
Stage: Stage-0
@@ -1691,9 +2166,27 @@ STAGE PLANS:
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.acid_2l_part_sdpo_no_cp
- Write Type: UPDATE
+ Write Type: DELETE
- Stage: Stage-3
+ Stage: Stage-4
+ Stats Work
+ Basic Stats Work:
+
+ Stage: Stage-1
+ Move Operator
+ tables:
+ partition:
+ ds
+ hr
+ replace: false
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.acid_2l_part_sdpo_no_cp
+ Write Type: INSERT
+
+ Stage: Stage-5
Stats Work
Basic Stats Work:
@@ -1701,12 +2194,17 @@ PREHOOK: query: update acid_2L_part_sdpo_no_cp set value = 'bar' where key = 'fo
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_2l_part_sdpo_no_cp
PREHOOK: Input: default@acid_2l_part_sdpo_no_cp@ds=2008-04-08/hr=11
+PREHOOK: Output: default@acid_2l_part_sdpo_no_cp
PREHOOK: Output: default@acid_2l_part_sdpo_no_cp@ds=2008-04-08/hr=11
POSTHOOK: query: update acid_2L_part_sdpo_no_cp set value = 'bar' where key = 'foo' and ds='2008-04-08' and hr=11
POSTHOOK: type: QUERY
POSTHOOK: Input: default@acid_2l_part_sdpo_no_cp
POSTHOOK: Input: default@acid_2l_part_sdpo_no_cp@ds=2008-04-08/hr=11
+POSTHOOK: Output: default@acid_2l_part_sdpo_no_cp
POSTHOOK: Output: default@acid_2l_part_sdpo_no_cp@ds=2008-04-08/hr=11
+POSTHOOK: Output: default@acid_2l_part_sdpo_no_cp@ds=2008-04-08/hr=11
+POSTHOOK: Lineage: acid_2l_part_sdpo_no_cp PARTITION(ds=2008-04-08,hr=11).key SIMPLE []
+POSTHOOK: Lineage: acid_2l_part_sdpo_no_cp PARTITION(ds=2008-04-08,hr=11).value SIMPLE []
PREHOOK: query: select count(*) from acid_2L_part_sdpo_no_cp where ds='2008-04-08' and hr=11
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_2l_part_sdpo_no_cp
@@ -1723,6 +2221,7 @@ PREHOOK: type: QUERY
PREHOOK: Input: default@acid_2l_part_sdpo_no_cp
PREHOOK: Input: default@acid_2l_part_sdpo_no_cp@ds=2008-04-08/hr=11
PREHOOK: Input: default@acid_2l_part_sdpo_no_cp@ds=2008-04-08/hr=12
+PREHOOK: Output: default@acid_2l_part_sdpo_no_cp
PREHOOK: Output: default@acid_2l_part_sdpo_no_cp@ds=2008-04-08/hr=11
PREHOOK: Output: default@acid_2l_part_sdpo_no_cp@ds=2008-04-08/hr=12
POSTHOOK: query: explain update acid_2L_part_sdpo_no_cp set value = 'bar' where key = 'foo' and ds='2008-04-08' and hr>=11
@@ -1730,20 +2229,24 @@ POSTHOOK: type: QUERY
POSTHOOK: Input: default@acid_2l_part_sdpo_no_cp
POSTHOOK: Input: default@acid_2l_part_sdpo_no_cp@ds=2008-04-08/hr=11
POSTHOOK: Input: default@acid_2l_part_sdpo_no_cp@ds=2008-04-08/hr=12
+POSTHOOK: Output: default@acid_2l_part_sdpo_no_cp
POSTHOOK: Output: default@acid_2l_part_sdpo_no_cp@ds=2008-04-08/hr=11
POSTHOOK: Output: default@acid_2l_part_sdpo_no_cp@ds=2008-04-08/hr=12
STAGE DEPENDENCIES:
- Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
- Stage-0 depends on stages: Stage-2
- Stage-3 depends on stages: Stage-0
+ Stage-2 is a root stage
+ Stage-3 depends on stages: Stage-2
+ Stage-0 depends on stages: Stage-3
+ Stage-4 depends on stages: Stage-0
+ Stage-1 depends on stages: Stage-3
+ Stage-5 depends on stages: Stage-1
STAGE PLANS:
- Stage: Stage-1
+ Stage: Stage-2
Tez
#### A masked pattern was here ####
Edges:
Reducer 2 <- Map 1 (SIMPLE_EDGE)
+ Reducer 3 <- Map 1 (SIMPLE_EDGE)
#### A masked pattern was here ####
Vertices:
Map 1
@@ -1759,33 +2262,63 @@ STAGE PLANS:
expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), hr (type: int)
outputColumnNames: _col0, _col4
Statistics: Num rows: 10 Data size: 800 Basic stats: COMPLETE Column stats: PARTIAL
- Reduce Output Operator
- key expressions: '2008-04-08' (type: string), _col4 (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), _col4 (type: int)
- Statistics: Num rows: 10 Data size: 3480 Basic stats: COMPLETE Column stats: PARTIAL
- value expressions: 'foo' (type: string), 'bar' (type: string)
+ 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)
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Statistics: Num rows: 10 Data size: 2720 Basic stats: COMPLETE Column stats: PARTIAL
+ Reduce Output Operator
+ key expressions: _col2 (type: string), _col3 (type: int), _bucket_number (type: string), _col0 (type: string)
+ null sort order: aaaa
+ sort order: ++++
+ Map-reduce partition columns: _col2 (type: string), _col3 (type: int)
+ Statistics: Num rows: 10 Data size: 2720 Basic stats: COMPLETE Column stats: PARTIAL
+ value expressions: _col1 (type: string)
Execution mode: llap
LLAP IO: may be used (ACID table)
Reducer 2
Execution mode: llap
Reduce Operator Tree:
Select Operator
- expressions: KEY._col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), VALUE._col1 (type: string), VALUE._col2 (type: string), KEY._col3 (type: string), KEY._col4 (type: int), KEY._bucket_number (type: string)
- outputColumnNames: _col0, _col1, _col2, _col3, _col4, _bucket_number
+ expressions: KEY._col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), KEY._col1 (type: string), KEY._col2 (type: int), KEY._bucket_number (type: string)
+ outputColumnNames: _col0, _col1, _col2, _bucket_number
+ File Output Operator
+ compressed: false
+ Dp Sort State: PARTITION_BUCKET_SORTED
+ Statistics: Num rows: 10 Data size: 1740 Basic stats: COMPLETE Column stats: PARTIAL
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.acid_2l_part_sdpo_no_cp
+ Write Type: DELETE
+ Reducer 3
+ Execution mode: llap
+ Reduce Operator Tree:
+ Select Operator
+ expressions: KEY._col0 (type: string), VALUE._col1 (type: string), KEY._col2 (type: string), KEY._col3 (type: int), KEY._bucket_number (type: string)
+ outputColumnNames: _col0, _col1, _col2, _col3, _bucket_number
File Output Operator
compressed: false
Dp Sort State: PARTITION_BUCKET_SORTED
- Statistics: Num rows: 10 Data size: 3480 Basic stats: COMPLETE Column stats: PARTIAL
+ Statistics: Num rows: 10 Data size: 2720 Basic stats: COMPLETE Column stats: PARTIAL
table:
input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.acid_2l_part_sdpo_no_cp
- Write Type: UPDATE
+ Write Type: INSERT
- Stage: Stage-2
+ Stage: Stage-3
Dependency Collection
Stage: Stage-0
@@ -1800,9 +2333,27 @@ STAGE PLANS:
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.acid_2l_part_sdpo_no_cp
- Write Type: UPDATE
+ Write Type: DELETE
- Stage: Stage-3
+ Stage: Stage-4
+ Stats Work
+ Basic Stats Work:
+
+ Stage: Stage-1
+ Move Operator
+ tables:
+ partition:
+ ds
+ hr
+ replace: false
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.acid_2l_part_sdpo_no_cp
+ Write Type: INSERT
+
+ Stage: Stage-5
Stats Work
Basic Stats Work:
@@ -1811,6 +2362,7 @@ PREHOOK: type: QUERY
PREHOOK: Input: default@acid_2l_part_sdpo_no_cp
PREHOOK: Input: default@acid_2l_part_sdpo_no_cp@ds=2008-04-08/hr=11
PREHOOK: Input: default@acid_2l_part_sdpo_no_cp@ds=2008-04-08/hr=12
+PREHOOK: Output: default@acid_2l_part_sdpo_no_cp
PREHOOK: Output: default@acid_2l_part_sdpo_no_cp@ds=2008-04-08/hr=11
PREHOOK: Output: default@acid_2l_part_sdpo_no_cp@ds=2008-04-08/hr=12
POSTHOOK: query: update acid_2L_part_sdpo_no_cp set value = 'bar' where key = 'foo' and ds='2008-04-08' and hr>=11
@@ -1818,8 +2370,12 @@ POSTHOOK: type: QUERY
POSTHOOK: Input: default@acid_2l_part_sdpo_no_cp
POSTHOOK: Input: default@acid_2l_part_sdpo_no_cp@ds=2008-04-08/hr=11
POSTHOOK: Input: default@acid_2l_part_sdpo_no_cp@ds=2008-04-08/hr=12
+POSTHOOK: Output: default@acid_2l_part_sdpo_no_cp
+POSTHOOK: Output: default@acid_2l_part_sdpo_no_cp@ds=2008-04-08/hr=11
POSTHOOK: Output: default@acid_2l_part_sdpo_no_cp@ds=2008-04-08/hr=11
POSTHOOK: Output: default@acid_2l_part_sdpo_no_cp@ds=2008-04-08/hr=12
+POSTHOOK: Lineage: acid_2l_part_sdpo_no_cp PARTITION(ds=2008-04-08,hr=11).key SIMPLE []
+POSTHOOK: Lineage: acid_2l_part_sdpo_no_cp PARTITION(ds=2008-04-08,hr=11).value SIMPLE []
PREHOOK: query: select count(*) from acid_2L_part_sdpo_no_cp where ds='2008-04-08' and hr>=11
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_2l_part_sdpo_no_cp
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 ada8b1c44e1..4a6324d7845 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
@@ -3428,22 +3428,27 @@ PREHOOK: query: explain update acid_uami_n1 set de = 3.14 where de = 109.23 or d
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_uami_n1
PREHOOK: Output: default@acid_uami_n1
+PREHOOK: Output: default@acid_uami_n1
POSTHOOK: query: explain update acid_uami_n1 set de = 3.14 where de = 109.23 or de = 119.23
POSTHOOK: type: QUERY
POSTHOOK: Input: default@acid_uami_n1
POSTHOOK: Output: default@acid_uami_n1
+POSTHOOK: Output: default@acid_uami_n1
STAGE DEPENDENCIES:
- Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
- Stage-0 depends on stages: Stage-2
- Stage-3 depends on stages: Stage-0
+ Stage-2 is a root stage
+ Stage-3 depends on stages: Stage-2
+ Stage-0 depends on stages: Stage-3
+ Stage-4 depends on stages: Stage-0
+ Stage-1 depends on stages: Stage-3
+ Stage-5 depends on stages: Stage-1
STAGE PLANS:
- Stage: Stage-1
+ Stage: Stage-2
Tez
#### A masked pattern was here ####
Edges:
Reducer 2 <- Map 1 (SIMPLE_EDGE)
+ Reducer 3 <- Map 1 (SIMPLE_EDGE)
#### A masked pattern was here ####
Vertices:
Map 1
@@ -3459,33 +3464,70 @@ STAGE PLANS:
expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), i (type: int), vc (type: varchar(128))
outputColumnNames: _col0, _col1, _col3
Statistics: Num rows: 6 Data size: 1134 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: 6 Data size: 1806 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: _col1 (type: int), 3.14 (type: decimal(5,2)), _col3 (type: varchar(128))
+ Select Operator
+ expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ outputColumnNames: _col0
+ Statistics: Num rows: 6 Data size: 456 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: 6 Data size: 456 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: _col1 (type: int), 3.14 (type: decimal(3,2)), _col3 (type: varchar(128))
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 6 Data size: 1350 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: enforce_constraint((_col1 is not null and _col2 is not null)) (type: boolean)
+ Statistics: Num rows: 3 Data size: 675 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: _col0 (type: int), CAST( _col1 AS decimal(5,2)) (type: decimal(5,2)), _col2 (type: varchar(128))
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 3 Data size: 675 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: 3 Data size: 675 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col1 (type: decimal(5,2)), _col2 (type: varchar(128))
Execution mode: vectorized, llap
LLAP IO: may be used (ACID table)
Reducer 2
Execution mode: vectorized, llap
Reduce Operator Tree:
Select Operator
- expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), VALUE._col0 (type: int), VALUE._col1 (type: decimal(5,2)), VALUE._col2 (type: varchar(128))
- outputColumnNames: _col0, _col1, _col2, _col3
- Statistics: Num rows: 6 Data size: 1806 Basic stats: COMPLETE Column stats: COMPLETE
+ expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ outputColumnNames: _col0
+ Statistics: Num rows: 6 Data size: 456 Basic stats: COMPLETE Column stats: COMPLETE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 6 Data size: 456 Basic stats: COMPLETE Column stats: COMPLETE
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.acid_uami_n1
+ Write Type: DELETE
+ Reducer 3
+ Execution mode: vectorized, llap
+ Reduce Operator Tree:
+ Select Operator
+ expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: decimal(5,2)), VALUE._col1 (type: varchar(128))
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 3 Data size: 675 Basic stats: COMPLETE Column stats: COMPLETE
File Output Operator
compressed: false
- Statistics: Num rows: 6 Data size: 1806 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 3 Data size: 675 Basic stats: COMPLETE Column stats: COMPLETE
table:
input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.acid_uami_n1
- Write Type: UPDATE
+ Write Type: INSERT
- Stage: Stage-2
+ Stage: Stage-3
Dependency Collection
Stage: Stage-0
@@ -3497,9 +3539,24 @@ STAGE PLANS:
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.acid_uami_n1
- Write Type: UPDATE
+ Write Type: DELETE
+
+ Stage: Stage-4
+ Stats Work
+ Basic Stats Work:
+
+ Stage: Stage-1
+ Move Operator
+ tables:
+ replace: false
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.acid_uami_n1
+ Write Type: INSERT
- Stage: Stage-3
+ Stage: Stage-5
Stats Work
Basic Stats Work:
@@ -3507,10 +3564,15 @@ PREHOOK: query: update acid_uami_n1 set de = 3.14 where de = 109.23 or de = 119.
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_uami_n1
PREHOOK: Output: default@acid_uami_n1
+PREHOOK: Output: default@acid_uami_n1
POSTHOOK: query: update acid_uami_n1 set de = 3.14 where de = 109.23 or de = 119.23
POSTHOOK: type: QUERY
POSTHOOK: Input: default@acid_uami_n1
POSTHOOK: Output: default@acid_uami_n1
+POSTHOOK: Output: default@acid_uami_n1
+POSTHOOK: Lineage: acid_uami_n1.de EXPRESSION []
+POSTHOOK: Lineage: acid_uami_n1.i SIMPLE [(acid_uami_n1)acid_uami_n1.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: acid_uami_n1.vc SIMPLE [(acid_uami_n1)acid_uami_n1.FieldSchema(name:vc, type:varchar(128), comment:null), ]
PREHOOK: query: ALTER table acid_uami_n1 drop constraint nn1
PREHOOK: type: ALTERTABLE_DROPCONSTRAINT
PREHOOK: Input: default@acid_uami_n1
@@ -3531,22 +3593,27 @@ PREHOOK: query: explain update acid_uami_n1 set de = 3.14159 where de = 3.14
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_uami_n1
PREHOOK: Output: default@acid_uami_n1
+PREHOOK: Output: default@acid_uami_n1
POSTHOOK: query: explain update acid_uami_n1 set de = 3.14159 where de = 3.14
POSTHOOK: type: QUERY
POSTHOOK: Input: default@acid_uami_n1
POSTHOOK: Output: default@acid_uami_n1
+POSTHOOK: Output: default@acid_uami_n1
STAGE DEPENDENCIES:
- Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
- Stage-0 depends on stages: Stage-2
- Stage-3 depends on stages: Stage-0
+ Stage-2 is a root stage
+ Stage-3 depends on stages: Stage-2
+ Stage-0 depends on stages: Stage-3
+ Stage-4 depends on stages: Stage-0
+ Stage-1 depends on stages: Stage-3
+ Stage-5 depends on stages: Stage-1
STAGE PLANS:
- Stage: Stage-1
+ Stage: Stage-2
Tez
#### A masked pattern was here ####
Edges:
Reducer 2 <- Map 1 (SIMPLE_EDGE)
+ Reducer 3 <- Map 1 (SIMPLE_EDGE)
#### A masked pattern was here ####
Vertices:
Map 1
@@ -3562,33 +3629,70 @@ STAGE PLANS:
expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), i (type: int), vc (type: varchar(128))
outputColumnNames: _col0, _col1, _col3
Statistics: Num rows: 3 Data size: 567 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: 3 Data size: 903 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: _col1 (type: int), 3.14 (type: decimal(5,2)), _col3 (type: varchar(128))
+ Select Operator
+ expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ outputColumnNames: _col0
+ Statistics: Num rows: 3 Data size: 228 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: 3 Data size: 228 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: _col1 (type: int), 3.14159 (type: decimal(6,5)), _col3 (type: varchar(128))
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 3 Data size: 675 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: enforce_constraint((_col0 is not null and _col2 is not null)) (type: boolean)
+ Statistics: Num rows: 1 Data size: 225 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: _col0 (type: int), CAST( _col1 AS decimal(5,2)) (type: decimal(5,2)), _col2 (type: varchar(128))
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 1 Data size: 225 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: 225 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col1 (type: decimal(5,2)), _col2 (type: varchar(128))
Execution mode: vectorized, llap
LLAP IO: may be used (ACID table)
Reducer 2
Execution mode: vectorized, llap
Reduce Operator Tree:
Select Operator
- expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), VALUE._col0 (type: int), VALUE._col1 (type: decimal(5,2)), VALUE._col2 (type: varchar(128))
- outputColumnNames: _col0, _col1, _col2, _col3
- Statistics: Num rows: 3 Data size: 903 Basic stats: COMPLETE Column stats: COMPLETE
+ expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ outputColumnNames: _col0
+ Statistics: Num rows: 3 Data size: 228 Basic stats: COMPLETE Column stats: COMPLETE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 3 Data size: 228 Basic stats: COMPLETE Column stats: COMPLETE
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.acid_uami_n1
+ Write Type: DELETE
+ Reducer 3
+ Execution mode: vectorized, llap
+ Reduce Operator Tree:
+ Select Operator
+ expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: decimal(5,2)), VALUE._col1 (type: varchar(128))
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 1 Data size: 225 Basic stats: COMPLETE Column stats: COMPLETE
File Output Operator
compressed: false
- Statistics: Num rows: 3 Data size: 903 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 1 Data size: 225 Basic stats: COMPLETE Column stats: COMPLETE
table:
input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.acid_uami_n1
- Write Type: UPDATE
+ Write Type: INSERT
- Stage: Stage-2
+ Stage: Stage-3
Dependency Collection
Stage: Stage-0
@@ -3600,9 +3704,24 @@ STAGE PLANS:
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.acid_uami_n1
- Write Type: UPDATE
+ Write Type: DELETE
+
+ Stage: Stage-4
+ Stats Work
+ Basic Stats Work:
+
+ Stage: Stage-1
+ Move Operator
+ tables:
+ replace: false
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.acid_uami_n1
+ Write Type: INSERT
- Stage: Stage-3
+ Stage: Stage-5
Stats Work
Basic Stats Work:
@@ -3610,10 +3729,15 @@ PREHOOK: query: update acid_uami_n1 set de = 3.14159 where de = 3.14
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_uami_n1
PREHOOK: Output: default@acid_uami_n1
+PREHOOK: Output: default@acid_uami_n1
POSTHOOK: query: update acid_uami_n1 set de = 3.14159 where de = 3.14
POSTHOOK: type: QUERY
POSTHOOK: Input: default@acid_uami_n1
POSTHOOK: Output: default@acid_uami_n1
+POSTHOOK: Output: default@acid_uami_n1
+POSTHOOK: Lineage: acid_uami_n1.de EXPRESSION []
+POSTHOOK: Lineage: acid_uami_n1.i SIMPLE [(acid_uami_n1)acid_uami_n1.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: acid_uami_n1.vc SIMPLE [(acid_uami_n1)acid_uami_n1.FieldSchema(name:vc, type:varchar(128), comment:null), ]
PREHOOK: query: explain
from src
insert overwrite table acid_uami_n1 select cast(key as int), cast(key as decimal(5,2)), value where key < 10
diff --git a/ql/src/test/results/clientpositive/llap/explain_locks.q.out b/ql/src/test/results/clientpositive/llap/explain_locks.q.out
index 7386cdeceb4..451ab07bce8 100644
--- a/ql/src/test/results/clientpositive/llap/explain_locks.q.out
+++ b/ql/src/test/results/clientpositive/llap/explain_locks.q.out
@@ -61,6 +61,7 @@ PREHOOK: Input: default@target
PREHOOK: Input: default@target@p=1/q=2
PREHOOK: Input: default@target@p=1/q=3
PREHOOK: Input: default@target@p=2/q=2
+PREHOOK: Output: default@target
PREHOOK: Output: default@target@p=1/q=2
PREHOOK: Output: default@target@p=1/q=3
PREHOOK: Output: default@target@p=2/q=2
@@ -71,6 +72,7 @@ POSTHOOK: Input: default@target
POSTHOOK: Input: default@target@p=1/q=2
POSTHOOK: Input: default@target@p=1/q=3
POSTHOOK: Input: default@target@p=2/q=2
+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
@@ -82,34 +84,40 @@ default.target.p=2/q=2 -> SHARED_READ
default.target.p=2/q=2 -> EXCL_WRITE
default.target.p=1/q=3 -> EXCL_WRITE
default.target.p=1/q=2 -> EXCL_WRITE
+default.target -> SHARED_READ
PREHOOK: query: explain locks update source set b1 = 1 where p1 in (select t.q from target t where t.p=2)
PREHOOK: type: QUERY
PREHOOK: Input: default@source
PREHOOK: Input: default@target
PREHOOK: Input: default@target@p=2/q=2
PREHOOK: Output: default@source
+PREHOOK: Output: default@source
POSTHOOK: query: explain locks update source set b1 = 1 where p1 in (select t.q from target t where t.p=2)
POSTHOOK: type: QUERY
POSTHOOK: Input: default@source
POSTHOOK: Input: default@target
POSTHOOK: Input: default@target@p=2/q=2
POSTHOOK: Output: default@source
+POSTHOOK: Output: default@source
LOCK INFORMATION:
default.target.p=2/q=2 -> SHARED_READ
default.source -> EXCL_WRITE
+default.source -> SHARED_READ
PREHOOK: query: explain formatted locks update source set b1 = 1 where p1 in (select t.q from target t where t.p=2)
PREHOOK: type: QUERY
PREHOOK: Input: default@source
PREHOOK: Input: default@target
PREHOOK: Input: default@target@p=2/q=2
PREHOOK: Output: default@source
+PREHOOK: Output: default@source
POSTHOOK: query: explain formatted locks update source set b1 = 1 where p1 in (select t.q from target t where t.p=2)
POSTHOOK: type: QUERY
POSTHOOK: Input: default@source
POSTHOOK: Input: default@target
POSTHOOK: Input: default@target@p=2/q=2
POSTHOOK: Output: default@source
-{"LOCK INFORMATION:":"[default.target.p=2/q=2 -> SHARED_READ, default.source -> EXCL_WRITE]"}
+POSTHOOK: Output: default@source
+{"LOCK INFORMATION:":"[default.target.p=2/q=2 -> SHARED_READ, default.source -> EXCL_WRITE, default.source -> SHARED_READ]"}
PREHOOK: query: explain locks merge into target t using source s on t.a = s.a1 when matched and p = 1 and q = 2 then update set b = 1 when matched and p = 2 and q = 2 then delete when not matched and a1 > 100 then insert values(s.a1,s.b1,s.p1, s.q1)
PREHOOK: type: QUERY
PREHOOK: Input: default@source
@@ -150,6 +158,7 @@ PREHOOK: Input: default@target
PREHOOK: Input: default@target@p=1/q=2
PREHOOK: Input: default@target@p=1/q=3
PREHOOK: Input: default@target@p=2/q=2
+PREHOOK: Output: default@target
PREHOOK: Output: default@target@p=1/q=2
PREHOOK: Output: default@target@p=1/q=3
PREHOOK: Output: default@target@p=2/q=2
@@ -160,6 +169,7 @@ POSTHOOK: Input: default@target
POSTHOOK: Input: default@target@p=1/q=2
POSTHOOK: Input: default@target@p=1/q=3
POSTHOOK: Input: default@target@p=2/q=2
+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
@@ -171,34 +181,40 @@ default.target.p=2/q=2 -> SHARED_READ
default.target.p=2/q=2 -> SHARED_WRITE
default.target.p=1/q=3 -> SHARED_WRITE
default.target.p=1/q=2 -> SHARED_WRITE
+default.target -> SHARED_WRITE
PREHOOK: query: explain locks update source set b1 = 1 where p1 in (select t.q from target t where t.p=2)
PREHOOK: type: QUERY
PREHOOK: Input: default@source
PREHOOK: Input: default@target
PREHOOK: Input: default@target@p=2/q=2
PREHOOK: Output: default@source
+PREHOOK: Output: default@source
POSTHOOK: query: explain locks update source set b1 = 1 where p1 in (select t.q from target t where t.p=2)
POSTHOOK: type: QUERY
POSTHOOK: Input: default@source
POSTHOOK: Input: default@target
POSTHOOK: Input: default@target@p=2/q=2
POSTHOOK: Output: default@source
+POSTHOOK: Output: default@source
LOCK INFORMATION:
default.target.p=2/q=2 -> SHARED_READ
default.source -> SHARED_WRITE
+default.source -> SHARED_WRITE
PREHOOK: query: explain formatted locks update source set b1 = 1 where p1 in (select t.q from target t where t.p=2)
PREHOOK: type: QUERY
PREHOOK: Input: default@source
PREHOOK: Input: default@target
PREHOOK: Input: default@target@p=2/q=2
PREHOOK: Output: default@source
+PREHOOK: Output: default@source
POSTHOOK: query: explain formatted locks update source set b1 = 1 where p1 in (select t.q from target t where t.p=2)
POSTHOOK: type: QUERY
POSTHOOK: Input: default@source
POSTHOOK: Input: default@target
POSTHOOK: Input: default@target@p=2/q=2
POSTHOOK: Output: default@source
-{"LOCK INFORMATION:":"[default.target.p=2/q=2 -> SHARED_READ, default.source -> SHARED_WRITE]"}
+POSTHOOK: Output: default@source
+{"LOCK INFORMATION:":"[default.target.p=2/q=2 -> SHARED_READ, default.source -> SHARED_WRITE, default.source -> SHARED_WRITE]"}
PREHOOK: query: explain locks merge into target t using source s on t.a = s.a1 when matched and p = 1 and q = 2 then update set b = 1 when matched and p = 2 and q = 2 then delete when not matched and a1 > 100 then insert values(s.a1,s.b1,s.p1, s.q1)
PREHOOK: type: QUERY
PREHOOK: Input: default@source
diff --git a/ql/src/test/results/clientpositive/llap/explainanalyze_acid_with_direct_insert.q.out b/ql/src/test/results/clientpositive/llap/explainanalyze_acid_with_direct_insert.q.out
index 38bbf1d7c1b..d06bd281d7d 100644
--- a/ql/src/test/results/clientpositive/llap/explainanalyze_acid_with_direct_insert.q.out
+++ b/ql/src/test/results/clientpositive/llap/explainanalyze_acid_with_direct_insert.q.out
@@ -158,26 +158,32 @@ PREHOOK: query: update analyze_acid_table set id=11 where id=1
PREHOOK: type: QUERY
PREHOOK: Input: default@analyze_acid_table
PREHOOK: Output: default@analyze_acid_table
+PREHOOK: Output: default@analyze_acid_table
POSTHOOK: query: update analyze_acid_table set id=11 where id=1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@analyze_acid_table
POSTHOOK: Output: default@analyze_acid_table
+POSTHOOK: Output: default@analyze_acid_table
PREHOOK: query: explain analyze update analyze_acid_table set id=11 where id=1
PREHOOK: type: QUERY
PREHOOK: Input: default@analyze_acid_table
PREHOOK: Output: default@analyze_acid_table
+PREHOOK: Output: default@analyze_acid_table
POSTHOOK: query: explain analyze update analyze_acid_table set id=11 where id=1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@analyze_acid_table
POSTHOOK: Output: default@analyze_acid_table
+POSTHOOK: Output: default@analyze_acid_table
STAGE DEPENDENCIES:
- Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
- Stage-0 depends on stages: Stage-2
- Stage-3 depends on stages: Stage-0
+ Stage-2 is a root stage
+ Stage-3 depends on stages: Stage-2
+ Stage-0 depends on stages: Stage-3
+ Stage-4 depends on stages: Stage-0
+ Stage-1 depends on stages: Stage-3
+ Stage-5 depends on stages: Stage-1
STAGE PLANS:
- Stage: Stage-1
+ Stage: Stage-2
Tez
#### A masked pattern was here ####
Edges:
@@ -203,26 +209,39 @@ STAGE PLANS:
sort order: +
Map-reduce partition columns: UDFToInteger(_col0) (type: int)
Statistics: Num rows: 1/1 Data size: 76 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: 11 (type: int)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1/1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1/1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.analyze_acid_table
+ Write Type: INSERT
Execution mode: vectorized, llap
LLAP IO: may be used (ACID table)
Reducer 2
Execution mode: vectorized, llap
Reduce Operator Tree:
Select Operator
- expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), 11 (type: int)
- outputColumnNames: _col0, _col1
- Statistics: Num rows: 1/1 Data size: 80 Basic stats: COMPLETE Column stats: COMPLETE
+ expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1/1 Data size: 76 Basic stats: COMPLETE Column stats: COMPLETE
File Output Operator
compressed: false
- Statistics: Num rows: 1/1 Data size: 80 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 1/1 Data size: 76 Basic stats: COMPLETE Column stats: COMPLETE
table:
input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.analyze_acid_table
- Write Type: UPDATE
+ Write Type: DELETE
- Stage: Stage-2
+ Stage: Stage-3
Dependency Collection
Stage: Stage-0
@@ -234,9 +253,24 @@ STAGE PLANS:
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.analyze_acid_table
- Write Type: UPDATE
+ Write Type: DELETE
- Stage: Stage-3
+ Stage: Stage-4
+ Stats Work
+ Basic Stats Work:
+
+ Stage: Stage-1
+ Move Operator
+ tables:
+ replace: false
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.analyze_acid_table
+ Write Type: INSERT
+
+ Stage: Stage-5
Stats Work
Basic Stats Work:
@@ -256,10 +290,13 @@ PREHOOK: query: update analyze_acid_table set id=22 where id=2
PREHOOK: type: QUERY
PREHOOK: Input: default@analyze_acid_table
PREHOOK: Output: default@analyze_acid_table
+PREHOOK: Output: default@analyze_acid_table
POSTHOOK: query: update analyze_acid_table set id=22 where id=2
POSTHOOK: type: QUERY
POSTHOOK: Input: default@analyze_acid_table
POSTHOOK: Output: default@analyze_acid_table
+POSTHOOK: Output: default@analyze_acid_table
+POSTHOOK: Lineage: analyze_acid_table.id SIMPLE []
PREHOOK: query: select * from analyze_acid_table
PREHOOK: type: QUERY
PREHOOK: Input: default@analyze_acid_table
@@ -601,30 +638,36 @@ PREHOOK: query: update analyze_part_table set a=11 where b=3
PREHOOK: type: QUERY
PREHOOK: Input: default@analyze_part_table
PREHOOK: Input: default@analyze_part_table@b=3
+PREHOOK: Output: default@analyze_part_table
PREHOOK: Output: default@analyze_part_table@b=3
POSTHOOK: query: update analyze_part_table set a=11 where b=3
POSTHOOK: type: QUERY
POSTHOOK: Input: default@analyze_part_table
POSTHOOK: Input: default@analyze_part_table@b=3
+POSTHOOK: Output: default@analyze_part_table
POSTHOOK: Output: default@analyze_part_table@b=3
PREHOOK: query: explain analyze update analyze_part_table set a=11 where b=3
PREHOOK: type: QUERY
PREHOOK: Input: default@analyze_part_table
PREHOOK: Input: default@analyze_part_table@b=3
+PREHOOK: Output: default@analyze_part_table
PREHOOK: Output: default@analyze_part_table@b=3
POSTHOOK: query: explain analyze update analyze_part_table set a=11 where b=3
POSTHOOK: type: QUERY
POSTHOOK: Input: default@analyze_part_table
POSTHOOK: Input: default@analyze_part_table@b=3
+POSTHOOK: Output: default@analyze_part_table
POSTHOOK: Output: default@analyze_part_table@b=3
STAGE DEPENDENCIES:
- Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
- Stage-0 depends on stages: Stage-2
- Stage-3 depends on stages: Stage-0
+ Stage-2 is a root stage
+ Stage-3 depends on stages: Stage-2
+ Stage-0 depends on stages: Stage-3
+ Stage-4 depends on stages: Stage-0
+ Stage-1 depends on stages: Stage-3
+ Stage-5 depends on stages: Stage-1
STAGE PLANS:
- Stage: Stage-1
+ Stage: Stage-2
Tez
#### A masked pattern was here ####
Edges:
@@ -647,14 +690,27 @@ STAGE PLANS:
sort order: +
Map-reduce partition columns: UDFToInteger(_col0) (type: int)
Statistics: Num rows: 3/3 Data size: 6374 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: 11 (type: int), 3 (type: int)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 3/3 Data size: 6374 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 3/3 Data size: 6374 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.analyze_part_table
+ Write Type: INSERT
Execution mode: vectorized, llap
LLAP IO: may be used (ACID table)
Reducer 2
Execution mode: vectorized, llap
Reduce Operator Tree:
Select Operator
- expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), 11 (type: int), 3 (type: int)
- outputColumnNames: _col0, _col1, _col2
+ expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), 3 (type: int)
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 3/3 Data size: 6374 Basic stats: COMPLETE Column stats: NONE
File Output Operator
compressed: false
@@ -664,9 +720,9 @@ STAGE PLANS:
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.analyze_part_table
- Write Type: UPDATE
+ Write Type: DELETE
- Stage: Stage-2
+ Stage: Stage-3
Dependency Collection
Stage: Stage-0
@@ -680,9 +736,26 @@ STAGE PLANS:
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.analyze_part_table
- Write Type: UPDATE
+ Write Type: DELETE
- Stage: Stage-3
+ Stage: Stage-4
+ Stats Work
+ Basic Stats Work:
+
+ Stage: Stage-1
+ Move Operator
+ tables:
+ partition:
+ b
+ replace: false
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.analyze_part_table
+ Write Type: INSERT
+
+ Stage: Stage-5
Stats Work
Basic Stats Work:
@@ -828,12 +901,16 @@ PREHOOK: query: update analyze_part_table set a=22 where b=3
PREHOOK: type: QUERY
PREHOOK: Input: default@analyze_part_table
PREHOOK: Input: default@analyze_part_table@b=3
+PREHOOK: Output: default@analyze_part_table
PREHOOK: Output: default@analyze_part_table@b=3
POSTHOOK: query: update analyze_part_table set a=22 where b=3
POSTHOOK: type: QUERY
POSTHOOK: Input: default@analyze_part_table
POSTHOOK: Input: default@analyze_part_table@b=3
+POSTHOOK: Output: default@analyze_part_table
+POSTHOOK: Output: default@analyze_part_table@b=3
POSTHOOK: Output: default@analyze_part_table@b=3
+POSTHOOK: Lineage: analyze_part_table PARTITION(b=3).a SIMPLE []
PREHOOK: query: select * from analyze_part_table
PREHOOK: type: QUERY
PREHOOK: Input: default@analyze_part_table
@@ -1059,26 +1136,32 @@ PREHOOK: query: update analyze_acid_table set id=11 where id=1
PREHOOK: type: QUERY
PREHOOK: Input: default@analyze_acid_table
PREHOOK: Output: default@analyze_acid_table
+PREHOOK: Output: default@analyze_acid_table
POSTHOOK: query: update analyze_acid_table set id=11 where id=1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@analyze_acid_table
POSTHOOK: Output: default@analyze_acid_table
+POSTHOOK: Output: default@analyze_acid_table
PREHOOK: query: explain analyze update analyze_acid_table set id=11 where id=1
PREHOOK: type: QUERY
PREHOOK: Input: default@analyze_acid_table
PREHOOK: Output: default@analyze_acid_table
+PREHOOK: Output: default@analyze_acid_table
POSTHOOK: query: explain analyze update analyze_acid_table set id=11 where id=1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@analyze_acid_table
POSTHOOK: Output: default@analyze_acid_table
+POSTHOOK: Output: default@analyze_acid_table
STAGE DEPENDENCIES:
- Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
- Stage-0 depends on stages: Stage-2
- Stage-3 depends on stages: Stage-0
+ Stage-2 is a root stage
+ Stage-3 depends on stages: Stage-2
+ Stage-0 depends on stages: Stage-3
+ Stage-4 depends on stages: Stage-0
+ Stage-1 depends on stages: Stage-3
+ Stage-5 depends on stages: Stage-1
STAGE PLANS:
- Stage: Stage-1
+ Stage: Stage-2
Tez
#### A masked pattern was here ####
Edges:
@@ -1104,26 +1187,39 @@ STAGE PLANS:
sort order: +
Map-reduce partition columns: UDFToInteger(_col0) (type: int)
Statistics: Num rows: 1/1 Data size: 76 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: 11 (type: int)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1/1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1/1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.analyze_acid_table
+ Write Type: INSERT
Execution mode: vectorized, llap
LLAP IO: may be used (ACID table)
Reducer 2
Execution mode: vectorized, llap
Reduce Operator Tree:
Select Operator
- expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), 11 (type: int)
- outputColumnNames: _col0, _col1
- Statistics: Num rows: 1/1 Data size: 80 Basic stats: COMPLETE Column stats: COMPLETE
+ expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1/1 Data size: 76 Basic stats: COMPLETE Column stats: COMPLETE
File Output Operator
compressed: false
- Statistics: Num rows: 1/1 Data size: 80 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 1/1 Data size: 76 Basic stats: COMPLETE Column stats: COMPLETE
table:
input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.analyze_acid_table
- Write Type: UPDATE
+ Write Type: DELETE
- Stage: Stage-2
+ Stage: Stage-3
Dependency Collection
Stage: Stage-0
@@ -1135,9 +1231,24 @@ STAGE PLANS:
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.analyze_acid_table
- Write Type: UPDATE
+ Write Type: DELETE
- Stage: Stage-3
+ Stage: Stage-4
+ Stats Work
+ Basic Stats Work:
+
+ Stage: Stage-1
+ Move Operator
+ tables:
+ replace: false
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.analyze_acid_table
+ Write Type: INSERT
+
+ Stage: Stage-5
Stats Work
Basic Stats Work:
@@ -1157,10 +1268,13 @@ PREHOOK: query: update analyze_acid_table set id=22 where id=2
PREHOOK: type: QUERY
PREHOOK: Input: default@analyze_acid_table
PREHOOK: Output: default@analyze_acid_table
+PREHOOK: Output: default@analyze_acid_table
POSTHOOK: query: update analyze_acid_table set id=22 where id=2
POSTHOOK: type: QUERY
POSTHOOK: Input: default@analyze_acid_table
POSTHOOK: Output: default@analyze_acid_table
+POSTHOOK: Output: default@analyze_acid_table
+POSTHOOK: Lineage: analyze_acid_table.id SIMPLE []
PREHOOK: query: select * from analyze_acid_table
PREHOOK: type: QUERY
PREHOOK: Input: default@analyze_acid_table
@@ -1484,30 +1598,36 @@ PREHOOK: query: update analyze_part_table set a=11 where b=3
PREHOOK: type: QUERY
PREHOOK: Input: default@analyze_part_table
PREHOOK: Input: default@analyze_part_table@b=3
+PREHOOK: Output: default@analyze_part_table
PREHOOK: Output: default@analyze_part_table@b=3
POSTHOOK: query: update analyze_part_table set a=11 where b=3
POSTHOOK: type: QUERY
POSTHOOK: Input: default@analyze_part_table
POSTHOOK: Input: default@analyze_part_table@b=3
+POSTHOOK: Output: default@analyze_part_table
POSTHOOK: Output: default@analyze_part_table@b=3
PREHOOK: query: explain analyze update analyze_part_table set a=11 where b=3
PREHOOK: type: QUERY
PREHOOK: Input: default@analyze_part_table
PREHOOK: Input: default@analyze_part_table@b=3
+PREHOOK: Output: default@analyze_part_table
PREHOOK: Output: default@analyze_part_table@b=3
POSTHOOK: query: explain analyze update analyze_part_table set a=11 where b=3
POSTHOOK: type: QUERY
POSTHOOK: Input: default@analyze_part_table
POSTHOOK: Input: default@analyze_part_table@b=3
+POSTHOOK: Output: default@analyze_part_table
POSTHOOK: Output: default@analyze_part_table@b=3
STAGE DEPENDENCIES:
- Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
- Stage-0 depends on stages: Stage-2
- Stage-3 depends on stages: Stage-0
+ Stage-2 is a root stage
+ Stage-3 depends on stages: Stage-2
+ Stage-0 depends on stages: Stage-3
+ Stage-4 depends on stages: Stage-0
+ Stage-1 depends on stages: Stage-3
+ Stage-5 depends on stages: Stage-1
STAGE PLANS:
- Stage: Stage-1
+ Stage: Stage-2
Tez
#### A masked pattern was here ####
Edges:
@@ -1530,14 +1650,27 @@ STAGE PLANS:
sort order: +
Map-reduce partition columns: UDFToInteger(_col0) (type: int)
Statistics: Num rows: 3/3 Data size: 6374 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: 11 (type: int), 3 (type: int)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 3/3 Data size: 6374 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 3/3 Data size: 6374 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.analyze_part_table
+ Write Type: INSERT
Execution mode: vectorized, llap
LLAP IO: may be used (ACID table)
Reducer 2
Execution mode: vectorized, llap
Reduce Operator Tree:
Select Operator
- expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), 11 (type: int), 3 (type: int)
- outputColumnNames: _col0, _col1, _col2
+ expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), 3 (type: int)
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 3/3 Data size: 6374 Basic stats: COMPLETE Column stats: NONE
File Output Operator
compressed: false
@@ -1547,9 +1680,9 @@ STAGE PLANS:
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.analyze_part_table
- Write Type: UPDATE
+ Write Type: DELETE
- Stage: Stage-2
+ Stage: Stage-3
Dependency Collection
Stage: Stage-0
@@ -1563,9 +1696,26 @@ STAGE PLANS:
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.analyze_part_table
- Write Type: UPDATE
+ Write Type: DELETE
- Stage: Stage-3
+ Stage: Stage-4
+ Stats Work
+ Basic Stats Work:
+
+ Stage: Stage-1
+ Move Operator
+ tables:
+ partition:
+ b
+ replace: false
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.analyze_part_table
+ Write Type: INSERT
+
+ Stage: Stage-5
Stats Work
Basic Stats Work:
@@ -1711,12 +1861,16 @@ PREHOOK: query: update analyze_part_table set a=22 where b=3
PREHOOK: type: QUERY
PREHOOK: Input: default@analyze_part_table
PREHOOK: Input: default@analyze_part_table@b=3
+PREHOOK: Output: default@analyze_part_table
PREHOOK: Output: default@analyze_part_table@b=3
POSTHOOK: query: update analyze_part_table set a=22 where b=3
POSTHOOK: type: QUERY
POSTHOOK: Input: default@analyze_part_table
POSTHOOK: Input: default@analyze_part_table@b=3
+POSTHOOK: Output: default@analyze_part_table
+POSTHOOK: Output: default@analyze_part_table@b=3
POSTHOOK: Output: default@analyze_part_table@b=3
+POSTHOOK: Lineage: analyze_part_table PARTITION(b=3).a SIMPLE []
PREHOOK: query: select * from analyze_part_table
PREHOOK: type: QUERY
PREHOOK: Input: default@analyze_part_table
diff --git a/ql/src/test/results/clientpositive/llap/fetch_deleted_rows.q.out b/ql/src/test/results/clientpositive/llap/fetch_deleted_rows.q.out
index 7d620a923af..a48805b6f59 100644
--- a/ql/src/test/results/clientpositive/llap/fetch_deleted_rows.q.out
+++ b/ql/src/test/results/clientpositive/llap/fetch_deleted_rows.q.out
@@ -54,12 +54,16 @@ where a = 3
PREHOOK: type: QUERY
PREHOOK: Input: default@t1
PREHOOK: Output: default@t1
+PREHOOK: Output: default@t1
POSTHOOK: query: update t1
set b = 'updated'
where a = 3
POSTHOOK: type: QUERY
POSTHOOK: Input: default@t1
POSTHOOK: Output: default@t1
+POSTHOOK: Output: default@t1
+POSTHOOK: Lineage: t1.a SIMPLE []
+POSTHOOK: Lineage: t1.b EXPRESSION []
PREHOOK: query: select t1.ROW__IS__DELETED, * from t1('acid.fetch.deleted.rows'='true') order by a
PREHOOK: type: QUERY
PREHOOK: Input: default@t1
diff --git a/ql/src/test/results/clientpositive/llap/fetch_deleted_rows_vector.q.out b/ql/src/test/results/clientpositive/llap/fetch_deleted_rows_vector.q.out
index e4fca112ea8..3f79aef961c 100644
--- a/ql/src/test/results/clientpositive/llap/fetch_deleted_rows_vector.q.out
+++ b/ql/src/test/results/clientpositive/llap/fetch_deleted_rows_vector.q.out
@@ -135,12 +135,16 @@ where a = 3
PREHOOK: type: QUERY
PREHOOK: Input: default@t1
PREHOOK: Output: default@t1
+PREHOOK: Output: default@t1
POSTHOOK: query: update t1
set b = 'updated'
where a = 3
POSTHOOK: type: QUERY
POSTHOOK: Input: default@t1
POSTHOOK: Output: default@t1
+POSTHOOK: Output: default@t1
+POSTHOOK: Lineage: t1.a SIMPLE []
+POSTHOOK: Lineage: t1.b EXPRESSION []
PREHOOK: query: select t1.ROW__IS__DELETED, * from t1('acid.fetch.deleted.rows'='true') order by a
PREHOOK: type: QUERY
PREHOOK: Input: default@t1
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 a8cdc187d91..d6bf75e4eaf 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
@@ -1987,22 +1987,27 @@ PREHOOK: query: EXPLAIN UPDATE insert_into1_n0 set key = DEFAULT where value=1
PREHOOK: type: QUERY
PREHOOK: Input: default@insert_into1_n0
PREHOOK: Output: default@insert_into1_n0
+PREHOOK: Output: default@insert_into1_n0
POSTHOOK: query: EXPLAIN UPDATE insert_into1_n0 set key = DEFAULT where value=1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@insert_into1_n0
POSTHOOK: Output: default@insert_into1_n0
+POSTHOOK: Output: default@insert_into1_n0
STAGE DEPENDENCIES:
- Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
- Stage-0 depends on stages: Stage-2
- Stage-3 depends on stages: Stage-0
+ Stage-2 is a root stage
+ Stage-3 depends on stages: Stage-2
+ Stage-0 depends on stages: Stage-3
+ Stage-4 depends on stages: Stage-0
+ Stage-1 depends on stages: Stage-3
+ Stage-5 depends on stages: Stage-1
STAGE PLANS:
- Stage: Stage-1
+ Stage: Stage-2
Tez
#### A masked pattern was here ####
Edges:
Reducer 2 <- Map 1 (SIMPLE_EDGE)
+ Reducer 3 <- Map 1 (SIMPLE_EDGE)
#### A masked pattern was here ####
Vertices:
Map 1
@@ -2018,33 +2023,63 @@ STAGE PLANS:
expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), value (type: string), i (type: int)
outputColumnNames: _col0, _col2, _col3
Statistics: Num rows: 1 Data size: 165 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: 169 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: 1 (type: int), _col2 (type: string), _col3 (type: int)
+ Select Operator
+ expressions: _col0 (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
+ Select Operator
+ expressions: 1 (type: int), _col2 (type: string), _col3 (type: int)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col2 (type: int)
+ null sort order: a
+ sort order: +
+ Map-reduce partition columns: _col2 (type: int)
+ Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col0 (type: int), _col1 (type: string)
Execution mode: vectorized, llap
LLAP IO: may be used (ACID table)
Reducer 2
Execution mode: vectorized, llap
Reduce Operator Tree:
Select Operator
- expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int)
- outputColumnNames: _col0, _col1, _col2, _col3
- Statistics: Num rows: 1 Data size: 169 Basic stats: COMPLETE Column stats: COMPLETE
+ expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: COMPLETE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: COMPLETE
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.insert_into1_n0
+ Write Type: DELETE
+ Reducer 3
+ Execution mode: vectorized, llap
+ Reduce Operator Tree:
+ Select Operator
+ expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), KEY.reducesinkkey0 (type: int)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
File Output Operator
compressed: false
- Statistics: Num rows: 1 Data size: 169 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
table:
input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.insert_into1_n0
- Write Type: UPDATE
+ Write Type: INSERT
- Stage: Stage-2
+ Stage: Stage-3
Dependency Collection
Stage: Stage-0
@@ -2056,9 +2091,24 @@ STAGE PLANS:
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.insert_into1_n0
- Write Type: UPDATE
+ Write Type: DELETE
- Stage: Stage-3
+ Stage: Stage-4
+ Stats Work
+ Basic Stats Work:
+
+ Stage: Stage-1
+ Move Operator
+ tables:
+ replace: false
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.insert_into1_n0
+ Write Type: INSERT
+
+ Stage: Stage-5
Stats Work
Basic Stats Work:
@@ -2066,10 +2116,15 @@ PREHOOK: query: UPDATE insert_into1_n0 set key = DEFAULT where value=1
PREHOOK: type: QUERY
PREHOOK: Input: default@insert_into1_n0
PREHOOK: Output: default@insert_into1_n0
+PREHOOK: Output: default@insert_into1_n0
POSTHOOK: query: UPDATE insert_into1_n0 set key = DEFAULT where value=1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@insert_into1_n0
POSTHOOK: Output: default@insert_into1_n0
+POSTHOOK: Output: default@insert_into1_n0
+POSTHOOK: Lineage: insert_into1_n0.i SIMPLE [(insert_into1_n0)insert_into1_n0.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: insert_into1_n0.key SIMPLE []
+POSTHOOK: Lineage: insert_into1_n0.value SIMPLE [(insert_into1_n0)insert_into1_n0.FieldSchema(name:value, type:string, comment:null), ]
PREHOOK: query: SELECT * from insert_into1_n0
PREHOOK: type: QUERY
PREHOOK: Input: default@insert_into1_n0
@@ -2100,22 +2155,27 @@ PREHOOK: query: EXPLAIN UPDATE insert_into1_n0 set key = DEFAULT, value=DEFAULT
PREHOOK: type: QUERY
PREHOOK: Input: default@insert_into1_n0
PREHOOK: Output: default@insert_into1_n0
+PREHOOK: Output: default@insert_into1_n0
POSTHOOK: query: EXPLAIN UPDATE insert_into1_n0 set key = DEFAULT, value=DEFAULT where value=1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@insert_into1_n0
POSTHOOK: Output: default@insert_into1_n0
+POSTHOOK: Output: default@insert_into1_n0
STAGE DEPENDENCIES:
- Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
- Stage-0 depends on stages: Stage-2
- Stage-3 depends on stages: Stage-0
+ Stage-2 is a root stage
+ Stage-3 depends on stages: Stage-2
+ Stage-0 depends on stages: Stage-3
+ Stage-4 depends on stages: Stage-0
+ Stage-1 depends on stages: Stage-3
+ Stage-5 depends on stages: Stage-1
STAGE PLANS:
- Stage: Stage-1
+ Stage: Stage-2
Tez
#### A masked pattern was here ####
Edges:
Reducer 2 <- Map 1 (SIMPLE_EDGE)
+ Reducer 3 <- Map 1 (SIMPLE_EDGE)
#### A masked pattern was here ####
Vertices:
Map 1
@@ -2131,33 +2191,63 @@ STAGE PLANS:
expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), i (type: int)
outputColumnNames: _col0, _col3
Statistics: Num rows: 1 Data size: 80 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: 168 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: 1 (type: int), null (type: string), _col3 (type: int)
+ Select Operator
+ expressions: _col0 (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
+ Select Operator
+ expressions: 1 (type: int), null (type: string), _col3 (type: int)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col2 (type: int)
+ null sort order: a
+ sort order: +
+ Map-reduce partition columns: _col2 (type: int)
+ Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col0 (type: int), _col1 (type: string)
Execution mode: vectorized, llap
LLAP IO: may be used (ACID table)
Reducer 2
Execution mode: vectorized, llap
Reduce Operator Tree:
Select Operator
- expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int)
- outputColumnNames: _col0, _col1, _col2, _col3
- Statistics: Num rows: 1 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
+ expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: COMPLETE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: COMPLETE
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.insert_into1_n0
+ Write Type: DELETE
+ Reducer 3
+ Execution mode: vectorized, llap
+ Reduce Operator Tree:
+ Select Operator
+ expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), KEY.reducesinkkey0 (type: int)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
File Output Operator
compressed: false
- Statistics: Num rows: 1 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
table:
input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.insert_into1_n0
- Write Type: UPDATE
+ Write Type: INSERT
- Stage: Stage-2
+ Stage: Stage-3
Dependency Collection
Stage: Stage-0
@@ -2169,9 +2259,24 @@ STAGE PLANS:
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.insert_into1_n0
- Write Type: UPDATE
+ Write Type: DELETE
- Stage: Stage-3
+ Stage: Stage-4
+ Stats Work
+ Basic Stats Work:
+
+ Stage: Stage-1
+ Move Operator
+ tables:
+ replace: false
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.insert_into1_n0
+ Write Type: INSERT
+
+ Stage: Stage-5
Stats Work
Basic Stats Work:
@@ -2179,10 +2284,15 @@ PREHOOK: query: UPDATE insert_into1_n0 set key = DEFAULT, value=DEFAULT where va
PREHOOK: type: QUERY
PREHOOK: Input: default@insert_into1_n0
PREHOOK: Output: default@insert_into1_n0
+PREHOOK: Output: default@insert_into1_n0
POSTHOOK: query: UPDATE insert_into1_n0 set key = DEFAULT, value=DEFAULT where value=1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@insert_into1_n0
POSTHOOK: Output: default@insert_into1_n0
+POSTHOOK: Output: default@insert_into1_n0
+POSTHOOK: Lineage: insert_into1_n0.i SIMPLE [(insert_into1_n0)insert_into1_n0.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: insert_into1_n0.key SIMPLE []
+POSTHOOK: Lineage: insert_into1_n0.value EXPRESSION []
PREHOOK: query: SELECT * from insert_into1_n0
PREHOOK: type: QUERY
PREHOOK: Input: default@insert_into1_n0
diff --git a/ql/src/test/results/clientpositive/llap/insert_into_default_keyword_2.q.out b/ql/src/test/results/clientpositive/llap/insert_into_default_keyword_2.q.out
index e74b210254a..302b80080f4 100644
--- a/ql/src/test/results/clientpositive/llap/insert_into_default_keyword_2.q.out
+++ b/ql/src/test/results/clientpositive/llap/insert_into_default_keyword_2.q.out
@@ -21,19 +21,23 @@ update t1 set a = `default`
PREHOOK: type: QUERY
PREHOOK: Input: default@t1
PREHOOK: Output: default@t1
+PREHOOK: Output: default@t1
POSTHOOK: query: explain
update t1 set a = `default`
POSTHOOK: type: QUERY
POSTHOOK: Input: default@t1
POSTHOOK: Output: default@t1
+POSTHOOK: Output: default@t1
STAGE DEPENDENCIES:
- Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
- Stage-0 depends on stages: Stage-2
- Stage-3 depends on stages: Stage-0
+ Stage-2 is a root stage
+ Stage-3 depends on stages: Stage-2
+ Stage-0 depends on stages: Stage-3
+ Stage-4 depends on stages: Stage-0
+ Stage-1 depends on stages: Stage-3
+ Stage-5 depends on stages: Stage-1
STAGE PLANS:
- Stage: Stage-1
+ Stage: Stage-2
Tez
#### A masked pattern was here ####
Edges:
@@ -46,36 +50,52 @@ STAGE PLANS:
alias: t1
Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
Select Operator
- expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), default (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: struct<writeid:bigint,bucketid:int,rowid:bigint>)
- null sort order: z
- sort order: +
- Map-reduce partition columns: UDFToInteger(_col0) (type: int)
- Statistics: Num rows: 2 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: _col1 (type: int)
+ expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), default (type: int), default (type: int)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ outputColumnNames: _col0
+ Statistics: Num rows: 2 Data size: 152 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: 2 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: _col1 (type: int), _col2 (type: int)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.t1
+ Write Type: INSERT
Execution mode: vectorized, llap
LLAP IO: may be used (ACID table)
Reducer 2
Execution mode: vectorized, llap
Reduce Operator Tree:
Select Operator
- expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), VALUE._col0 (type: int), VALUE._col0 (type: int)
- outputColumnNames: _col0, _col1, _col2
- Statistics: Num rows: 2 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE
+ expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ outputColumnNames: _col0
+ Statistics: Num rows: 2 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE
File Output Operator
compressed: false
- Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 2 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE
table:
input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.t1
- Write Type: UPDATE
+ Write Type: DELETE
- Stage: Stage-2
+ Stage: Stage-3
Dependency Collection
Stage: Stage-0
@@ -87,9 +107,24 @@ STAGE PLANS:
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.t1
- Write Type: UPDATE
+ Write Type: DELETE
- Stage: Stage-3
+ Stage: Stage-4
+ Stats Work
+ Basic Stats Work:
+
+ Stage: Stage-1
+ Move Operator
+ tables:
+ replace: false
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.t1
+ Write Type: INSERT
+
+ Stage: Stage-5
Stats Work
Basic Stats Work:
@@ -97,10 +132,14 @@ PREHOOK: query: update t1 set a = `default`
PREHOOK: type: QUERY
PREHOOK: Input: default@t1
PREHOOK: Output: default@t1
+PREHOOK: Output: default@t1
POSTHOOK: query: update t1 set a = `default`
POSTHOOK: type: QUERY
POSTHOOK: Input: default@t1
POSTHOOK: Output: default@t1
+POSTHOOK: Output: default@t1
+POSTHOOK: Lineage: t1.a SIMPLE [(t1)t1.FieldSchema(name:default, type:int, comment:null), ]
+POSTHOOK: Lineage: t1.default SIMPLE [(t1)t1.FieldSchema(name:default, type:int, comment:null), ]
PREHOOK: query: select * from t1
PREHOOK: type: QUERY
PREHOOK: Input: default@t1
diff --git a/ql/src/test/results/clientpositive/llap/insert_update_delete.q.out b/ql/src/test/results/clientpositive/llap/insert_update_delete.q.out
index 9a3cf4b5c21..54021a99c13 100644
--- a/ql/src/test/results/clientpositive/llap/insert_update_delete.q.out
+++ b/ql/src/test/results/clientpositive/llap/insert_update_delete.q.out
@@ -38,10 +38,14 @@ PREHOOK: query: update acid_iud set b = 'fred'
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_iud
PREHOOK: Output: default@acid_iud
+PREHOOK: Output: default@acid_iud
POSTHOOK: query: update acid_iud set b = 'fred'
POSTHOOK: type: QUERY
POSTHOOK: Input: default@acid_iud
POSTHOOK: Output: default@acid_iud
+POSTHOOK: Output: default@acid_iud
+POSTHOOK: Lineage: acid_iud.a SIMPLE [(acid_iud)acid_iud.FieldSchema(name:a, type:int, comment:null), ]
+POSTHOOK: Lineage: acid_iud.b EXPRESSION []
PREHOOK: query: select a,b from acid_iud order by a
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_iud
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 1d763a94435..6530a8f05b9 100644
--- a/ql/src/test/results/clientpositive/llap/llap_acid.q.out
+++ b/ql/src/test/results/clientpositive/llap/llap_acid.q.out
@@ -239,6 +239,7 @@ PREHOOK: Input: default@orc_llap_n1
PREHOOK: Input: default@orc_llap_n1@csmallint=1
PREHOOK: Input: default@orc_llap_n1@csmallint=2
PREHOOK: Input: default@orc_llap_n1@csmallint=3
+PREHOOK: Output: default@orc_llap_n1
PREHOOK: Output: default@orc_llap_n1@csmallint=1
PREHOOK: Output: default@orc_llap_n1@csmallint=2
PREHOOK: Output: default@orc_llap_n1@csmallint=3
@@ -249,6 +250,7 @@ POSTHOOK: Input: default@orc_llap_n1
POSTHOOK: Input: default@orc_llap_n1@csmallint=1
POSTHOOK: Input: default@orc_llap_n1@csmallint=2
POSTHOOK: Input: default@orc_llap_n1@csmallint=3
+POSTHOOK: Output: default@orc_llap_n1
POSTHOOK: Output: default@orc_llap_n1@csmallint=1
POSTHOOK: Output: default@orc_llap_n1@csmallint=2
POSTHOOK: Output: default@orc_llap_n1@csmallint=3
@@ -257,16 +259,19 @@ PLAN VECTORIZATION:
enabledConditionsMet: [hive.vectorized.execution.enabled IS true]
STAGE DEPENDENCIES:
- Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
- Stage-0 depends on stages: Stage-2
- Stage-3 depends on stages: Stage-0
+ Stage-2 is a root stage
+ Stage-3 depends on stages: Stage-2
+ Stage-0 depends on stages: Stage-3
+ Stage-4 depends on stages: Stage-0
+ Stage-1 depends on stages: Stage-3
+ Stage-5 depends on stages: Stage-1
STAGE PLANS:
- Stage: Stage-1
+ Stage: Stage-2
Tez
Edges:
Reducer 2 <- Map 1 (SIMPLE_EDGE)
+ Reducer 3 <- Map 1 (SIMPLE_EDGE)
Vertices:
Map 1
Map Operator Tree:
@@ -281,14 +286,29 @@ STAGE PLANS:
className: VectorSelectOperator
native: true
projectedOutputColumnNums: [5, 2, 3, 4]
- Reduce Sink Vectorization:
- className: VectorReduceSinkObjectHashOperator
- keyColumns: 5:struct<writeid:bigint,bucketid:int,rowid:bigint>
+ Select Vectorization:
+ className: VectorSelectOperator
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
- partitionColumns: 9:int
- valueColumns: 7:int, 8:bigint, 2:float, 3:double, 4:smallint
- valueExpressions: ConstantVectorExpression(val 1) -> 7:int, ConstantVectorExpression(val 2) -> 8:bigint
+ projectedOutputColumnNums: [5, 4]
+ Reduce Sink Vectorization:
+ className: VectorReduceSinkObjectHashOperator
+ keyColumns: 5:struct<writeid:bigint,bucketid:int,rowid:bigint>
+ 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
+ partitionColumns: 9:int
+ valueColumns: 4:smallint
+ Select Vectorization:
+ className: VectorSelectOperator
+ native: true
+ projectedOutputColumnNums: [7, 8, 2, 3, 4]
+ selectExpressions: ConstantVectorExpression(val 1) -> 7:int, ConstantVectorExpression(val 2) -> 8:bigint
+ Reduce Sink Vectorization:
+ className: VectorReduceSinkObjectHashOperator
+ keyColumns: 7:int
+ 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
+ partitionColumns: 7:int
+ valueColumns: 8:bigint, 2:float, 3:double, 4:smallint
Execution mode: vectorized, llap
LLAP IO: may be used (ACID table)
Map Vectorization:
@@ -319,24 +339,51 @@ STAGE PLANS:
usesVectorUDFAdaptor: false
vectorized: true
rowBatchContext:
- dataColumnCount: 6
- dataColumns: KEY.reducesinkkey0:struct<writeid:bigint,bucketid:int,rowid:bigint>, VALUE._col0:int, VALUE._col1:bigint, VALUE._col2:float, VALUE._col3:double, VALUE._col4:smallint
+ dataColumnCount: 2
+ dataColumns: KEY.reducesinkkey0:struct<writeid:bigint,bucketid:int,rowid:bigint>, VALUE._col0:smallint
+ partitionColumnCount: 0
+ scratchColumnTypeNames: []
+ Reduce Operator Tree:
+ Select Vectorization:
+ className: VectorSelectOperator
+ native: true
+ projectedOutputColumnNums: [0, 1]
+ File Sink Vectorization:
+ className: VectorFileSinkOperator
+ native: false
+ Reducer 3
+ Execution mode: vectorized, llap
+ Reduce Vectorization:
+ enabled: true
+ enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez] IS true
+ reduceColumnNullOrder: a
+ reduceColumnSortOrder: +
+ allNative: false
+ usesVectorUDFAdaptor: false
+ vectorized: true
+ rowBatchContext:
+ dataColumnCount: 5
+ dataColumns: KEY.reducesinkkey0:int, VALUE._col0:bigint, VALUE._col1:float, VALUE._col2:double, VALUE._col3:smallint
partitionColumnCount: 0
scratchColumnTypeNames: []
Reduce Operator Tree:
Select Vectorization:
className: VectorSelectOperator
native: true
- projectedOutputColumnNums: [0, 1, 2, 3, 4, 5]
+ projectedOutputColumnNums: [0, 1, 2, 3, 4]
File Sink Vectorization:
className: VectorFileSinkOperator
native: false
- Stage: Stage-2
+ Stage: Stage-3
Stage: Stage-0
- Stage: Stage-3
+ Stage: Stage-4
+
+ Stage: Stage-1
+
+ Stage: Stage-5
PREHOOK: query: update orc_llap_n1 set cbigint = 2 where cint = 1
PREHOOK: type: QUERY
@@ -344,6 +391,7 @@ PREHOOK: Input: default@orc_llap_n1
PREHOOK: Input: default@orc_llap_n1@csmallint=1
PREHOOK: Input: default@orc_llap_n1@csmallint=2
PREHOOK: Input: default@orc_llap_n1@csmallint=3
+PREHOOK: Output: default@orc_llap_n1
PREHOOK: Output: default@orc_llap_n1@csmallint=1
PREHOOK: Output: default@orc_llap_n1@csmallint=2
PREHOOK: Output: default@orc_llap_n1@csmallint=3
@@ -353,9 +401,15 @@ POSTHOOK: Input: default@orc_llap_n1
POSTHOOK: Input: default@orc_llap_n1@csmallint=1
POSTHOOK: Input: default@orc_llap_n1@csmallint=2
POSTHOOK: Input: default@orc_llap_n1@csmallint=3
+POSTHOOK: Output: default@orc_llap_n1
+POSTHOOK: Output: default@orc_llap_n1@csmallint=1
POSTHOOK: Output: default@orc_llap_n1@csmallint=1
POSTHOOK: Output: default@orc_llap_n1@csmallint=2
POSTHOOK: Output: default@orc_llap_n1@csmallint=3
+POSTHOOK: Lineage: orc_llap_n1 PARTITION(csmallint=1).cbigint EXPRESSION []
+POSTHOOK: Lineage: orc_llap_n1 PARTITION(csmallint=1).cdouble SIMPLE [(orc_llap_n1)orc_llap_n1.FieldSchema(name:cdouble, type:double, comment:null), ]
+POSTHOOK: Lineage: orc_llap_n1 PARTITION(csmallint=1).cfloat SIMPLE [(orc_llap_n1)orc_llap_n1.FieldSchema(name:cfloat, type:float, comment:null), ]
+POSTHOOK: Lineage: orc_llap_n1 PARTITION(csmallint=1).cint SIMPLE []
PREHOOK: query: explain vectorization only detail
select cint, csmallint, cbigint from orc_llap_n1 where cint is not null order
by csmallint, cint
diff --git a/ql/src/test/results/clientpositive/llap/llap_acid2.q.out b/ql/src/test/results/clientpositive/llap/llap_acid2.q.out
index eaae3e8bdf2..3dc4be65695 100644
--- a/ql/src/test/results/clientpositive/llap/llap_acid2.q.out
+++ b/ql/src/test/results/clientpositive/llap/llap_acid2.q.out
@@ -227,11 +227,29 @@ cdecimal2 = cast("9.987654321" as decimal(38,18)) where cstring1 = 'N016jPED08o
PREHOOK: type: QUERY
PREHOOK: Input: default@orc_llap2
PREHOOK: Output: default@orc_llap2
+PREHOOK: Output: default@orc_llap2
POSTHOOK: query: update orc_llap2 set cstring1 = 'testvalue', cdecimal1 = cast("3.321" as decimal(10,3)),
cdecimal2 = cast("9.987654321" as decimal(38,18)) where cstring1 = 'N016jPED08o'
POSTHOOK: type: QUERY
POSTHOOK: Input: default@orc_llap2
POSTHOOK: Output: default@orc_llap2
+POSTHOOK: Output: default@orc_llap2
+POSTHOOK: Lineage: orc_llap2.cbigint SIMPLE [(orc_llap2)orc_llap2.FieldSchema(name:cbigint, type:bigint, comment:null), ]
+POSTHOOK: Lineage: orc_llap2.cbigint0 SIMPLE [(orc_llap2)orc_llap2.FieldSchema(name:cbigint0, type:bigint, comment:null), ]
+POSTHOOK: Lineage: orc_llap2.cbigint1 SIMPLE [(orc_llap2)orc_llap2.FieldSchema(name:cbigint1, type:bigint, comment:null), ]
+POSTHOOK: Lineage: orc_llap2.cdecimal1 SIMPLE []
+POSTHOOK: Lineage: orc_llap2.cdecimal2 EXPRESSION []
+POSTHOOK: Lineage: orc_llap2.cdouble SIMPLE [(orc_llap2)orc_llap2.FieldSchema(name:cdouble, type:double, comment:null), ]
+POSTHOOK: Lineage: orc_llap2.cdouble0 SIMPLE [(orc_llap2)orc_llap2.FieldSchema(name:cdouble0, type:double, comment:null), ]
+POSTHOOK: Lineage: orc_llap2.cdouble1 SIMPLE [(orc_llap2)orc_llap2.FieldSchema(name:cdouble1, type:double, comment:null), ]
+POSTHOOK: Lineage: orc_llap2.cfloat SIMPLE [(orc_llap2)orc_llap2.FieldSchema(name:cfloat, type:float, comment:null), ]
+POSTHOOK: Lineage: orc_llap2.cfloat0 SIMPLE [(orc_llap2)orc_llap2.FieldSchema(name:cfloat0, type:float, comment:null), ]
+POSTHOOK: Lineage: orc_llap2.cfloat1 SIMPLE [(orc_llap2)orc_llap2.FieldSchema(name:cfloat1, type:float, comment:null), ]
+POSTHOOK: Lineage: orc_llap2.cfloat2 SIMPLE [(orc_llap2)orc_llap2.FieldSchema(name:cfloat2, type:float, comment:null), ]
+POSTHOOK: Lineage: orc_llap2.cint SIMPLE [(orc_llap2)orc_llap2.FieldSchema(name:cint, type:int, comment:null), ]
+POSTHOOK: Lineage: orc_llap2.cint0 SIMPLE [(orc_llap2)orc_llap2.FieldSchema(name:cint0, type:int, comment:null), ]
+POSTHOOK: Lineage: orc_llap2.cint1 SIMPLE [(orc_llap2)orc_llap2.FieldSchema(name:cint1, type:int, comment:null), ]
+POSTHOOK: Lineage: orc_llap2.cstring1 SIMPLE []
PREHOOK: query: select cstring1 from orc_llap_n2
PREHOOK: type: QUERY
PREHOOK: Input: default@orc_llap_n2
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 e361896d25c..8f6f948e940 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
@@ -233,6 +233,7 @@ PREHOOK: Input: default@orc_llap_acid_fast
PREHOOK: Input: default@orc_llap_acid_fast@csmallint=1
PREHOOK: Input: default@orc_llap_acid_fast@csmallint=2
PREHOOK: Input: default@orc_llap_acid_fast@csmallint=3
+PREHOOK: Output: default@orc_llap_acid_fast
PREHOOK: Output: default@orc_llap_acid_fast@csmallint=1
PREHOOK: Output: default@orc_llap_acid_fast@csmallint=2
PREHOOK: Output: default@orc_llap_acid_fast@csmallint=3
@@ -243,6 +244,7 @@ POSTHOOK: Input: default@orc_llap_acid_fast
POSTHOOK: Input: default@orc_llap_acid_fast@csmallint=1
POSTHOOK: Input: default@orc_llap_acid_fast@csmallint=2
POSTHOOK: Input: default@orc_llap_acid_fast@csmallint=3
+POSTHOOK: Output: default@orc_llap_acid_fast
POSTHOOK: Output: default@orc_llap_acid_fast@csmallint=1
POSTHOOK: Output: default@orc_llap_acid_fast@csmallint=2
POSTHOOK: Output: default@orc_llap_acid_fast@csmallint=3
@@ -251,16 +253,19 @@ PLAN VECTORIZATION:
enabledConditionsMet: [hive.vectorized.execution.enabled IS true]
STAGE DEPENDENCIES:
- Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
- Stage-0 depends on stages: Stage-2
- Stage-3 depends on stages: Stage-0
+ Stage-2 is a root stage
+ Stage-3 depends on stages: Stage-2
+ Stage-0 depends on stages: Stage-3
+ Stage-4 depends on stages: Stage-0
+ Stage-1 depends on stages: Stage-3
+ Stage-5 depends on stages: Stage-1
STAGE PLANS:
- Stage: Stage-1
+ Stage: Stage-2
Tez
Edges:
Reducer 2 <- Map 1 (SIMPLE_EDGE)
+ Reducer 3 <- Map 1 (SIMPLE_EDGE)
Vertices:
Map 1
Map Operator Tree:
@@ -275,14 +280,29 @@ STAGE PLANS:
className: VectorSelectOperator
native: true
projectedOutputColumnNums: [5, 2, 3, 4]
- Reduce Sink Vectorization:
- className: VectorReduceSinkObjectHashOperator
- keyColumns: 5:struct<writeid:bigint,bucketid:int,rowid:bigint>
+ Select Vectorization:
+ className: VectorSelectOperator
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
- partitionColumns: 9:int
- valueColumns: 7:int, 8:bigint, 2:float, 3:double, 4:smallint
- valueExpressions: ConstantVectorExpression(val 1) -> 7:int, ConstantVectorExpression(val 2) -> 8:bigint
+ projectedOutputColumnNums: [5, 4]
+ Reduce Sink Vectorization:
+ className: VectorReduceSinkObjectHashOperator
+ keyColumns: 5:struct<writeid:bigint,bucketid:int,rowid:bigint>
+ 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
+ partitionColumns: 9:int
+ valueColumns: 4:smallint
+ Select Vectorization:
+ className: VectorSelectOperator
+ native: true
+ projectedOutputColumnNums: [7, 8, 2, 3, 4]
+ selectExpressions: ConstantVectorExpression(val 1) -> 7:int, ConstantVectorExpression(val 2) -> 8:bigint
+ Reduce Sink Vectorization:
+ className: VectorReduceSinkObjectHashOperator
+ keyColumns: 7:int
+ 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
+ partitionColumns: 7:int
+ valueColumns: 8:bigint, 2:float, 3:double, 4:smallint
Execution mode: vectorized, llap
LLAP IO: may be used (ACID table)
Map Vectorization:
@@ -313,24 +333,51 @@ STAGE PLANS:
usesVectorUDFAdaptor: false
vectorized: true
rowBatchContext:
- dataColumnCount: 6
- dataColumns: KEY.reducesinkkey0:struct<writeid:bigint,bucketid:int,rowid:bigint>, VALUE._col0:int, VALUE._col1:bigint, VALUE._col2:float, VALUE._col3:double, VALUE._col4:smallint
+ dataColumnCount: 2
+ dataColumns: KEY.reducesinkkey0:struct<writeid:bigint,bucketid:int,rowid:bigint>, VALUE._col0:smallint
+ partitionColumnCount: 0
+ scratchColumnTypeNames: []
+ Reduce Operator Tree:
+ Select Vectorization:
+ className: VectorSelectOperator
+ native: true
+ projectedOutputColumnNums: [0, 1]
+ File Sink Vectorization:
+ className: VectorFileSinkOperator
+ native: false
+ Reducer 3
+ Execution mode: vectorized, llap
+ Reduce Vectorization:
+ enabled: true
+ enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez] IS true
+ reduceColumnNullOrder: a
+ reduceColumnSortOrder: +
+ allNative: false
+ usesVectorUDFAdaptor: false
+ vectorized: true
+ rowBatchContext:
+ dataColumnCount: 5
+ dataColumns: KEY.reducesinkkey0:int, VALUE._col0:bigint, VALUE._col1:float, VALUE._col2:double, VALUE._col3:smallint
partitionColumnCount: 0
scratchColumnTypeNames: []
Reduce Operator Tree:
Select Vectorization:
className: VectorSelectOperator
native: true
- projectedOutputColumnNums: [0, 1, 2, 3, 4, 5]
+ projectedOutputColumnNums: [0, 1, 2, 3, 4]
File Sink Vectorization:
className: VectorFileSinkOperator
native: false
- Stage: Stage-2
+ Stage: Stage-3
Stage: Stage-0
- Stage: Stage-3
+ Stage: Stage-4
+
+ Stage: Stage-1
+
+ Stage: Stage-5
PREHOOK: query: update orc_llap_acid_fast set cbigint = 2 where cint = 1
PREHOOK: type: QUERY
@@ -338,6 +385,7 @@ PREHOOK: Input: default@orc_llap_acid_fast
PREHOOK: Input: default@orc_llap_acid_fast@csmallint=1
PREHOOK: Input: default@orc_llap_acid_fast@csmallint=2
PREHOOK: Input: default@orc_llap_acid_fast@csmallint=3
+PREHOOK: Output: default@orc_llap_acid_fast
PREHOOK: Output: default@orc_llap_acid_fast@csmallint=1
PREHOOK: Output: default@orc_llap_acid_fast@csmallint=2
PREHOOK: Output: default@orc_llap_acid_fast@csmallint=3
@@ -347,9 +395,15 @@ POSTHOOK: Input: default@orc_llap_acid_fast
POSTHOOK: Input: default@orc_llap_acid_fast@csmallint=1
POSTHOOK: Input: default@orc_llap_acid_fast@csmallint=2
POSTHOOK: Input: default@orc_llap_acid_fast@csmallint=3
+POSTHOOK: Output: default@orc_llap_acid_fast
+POSTHOOK: Output: default@orc_llap_acid_fast@csmallint=1
POSTHOOK: Output: default@orc_llap_acid_fast@csmallint=1
POSTHOOK: Output: default@orc_llap_acid_fast@csmallint=2
POSTHOOK: Output: default@orc_llap_acid_fast@csmallint=3
+POSTHOOK: Lineage: orc_llap_acid_fast PARTITION(csmallint=1).cbigint EXPRESSION []
+POSTHOOK: Lineage: orc_llap_acid_fast PARTITION(csmallint=1).cdouble SIMPLE [(orc_llap_acid_fast)orc_llap_acid_fast.FieldSchema(name:cdouble, type:double, comment:null), ]
+POSTHOOK: Lineage: orc_llap_acid_fast PARTITION(csmallint=1).cfloat SIMPLE [(orc_llap_acid_fast)orc_llap_acid_fast.FieldSchema(name:cfloat, type:float, comment:null), ]
+POSTHOOK: Lineage: orc_llap_acid_fast PARTITION(csmallint=1).cint SIMPLE []
PREHOOK: query: explain vectorization only detail
select cint, csmallint, cbigint from orc_llap_acid_fast where cint is not null order
by csmallint, cint
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 1fb9acefd2a..7ce45a64f60 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
@@ -22,10 +22,14 @@ PREHOOK: query: update masking_acid_no_masking set key=1 where value='ddd'
PREHOOK: type: QUERY
PREHOOK: Input: default@masking_acid_no_masking
PREHOOK: Output: default@masking_acid_no_masking
+PREHOOK: Output: default@masking_acid_no_masking
POSTHOOK: query: update masking_acid_no_masking set key=1 where value='ddd'
POSTHOOK: type: QUERY
POSTHOOK: Input: default@masking_acid_no_masking
POSTHOOK: Output: default@masking_acid_no_masking
+POSTHOOK: Output: default@masking_acid_no_masking
+POSTHOOK: Lineage: masking_acid_no_masking.key SIMPLE []
+POSTHOOK: Lineage: masking_acid_no_masking.value SIMPLE []
PREHOOK: query: delete from masking_acid_no_masking where value='ddd'
PREHOOK: type: QUERY
PREHOOK: Input: default@masking_acid_no_masking
diff --git a/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_4.q.out b/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_4.q.out
index 9cc6af35524..4b8cd42ef9d 100644
--- a/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_4.q.out
+++ b/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_4.q.out
@@ -1122,10 +1122,16 @@ PREHOOK: query: UPDATE cmv_basetable_2_n2 SET a=2 WHERE a=1
PREHOOK: type: QUERY
PREHOOK: Input: default@cmv_basetable_2_n2
PREHOOK: Output: default@cmv_basetable_2_n2
+PREHOOK: Output: default@cmv_basetable_2_n2
POSTHOOK: query: UPDATE cmv_basetable_2_n2 SET a=2 WHERE a=1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@cmv_basetable_2_n2
POSTHOOK: Output: default@cmv_basetable_2_n2
+POSTHOOK: Output: default@cmv_basetable_2_n2
+POSTHOOK: Lineage: cmv_basetable_2_n2.a SIMPLE []
+POSTHOOK: Lineage: cmv_basetable_2_n2.b SIMPLE [(cmv_basetable_2_n2)cmv_basetable_2_n2.FieldSchema(name:b, type:varchar(256), comment:null), ]
+POSTHOOK: Lineage: cmv_basetable_2_n2.c SIMPLE [(cmv_basetable_2_n2)cmv_basetable_2_n2.FieldSchema(name:c, type:decimal(10,2), comment:null), ]
+POSTHOOK: Lineage: cmv_basetable_2_n2.d SIMPLE [(cmv_basetable_2_n2)cmv_basetable_2_n2.FieldSchema(name:d, type:int, comment:null), ]
PREHOOK: query: EXPLAIN
ALTER MATERIALIZED VIEW cmv_mat_view_n5 REBUILD
PREHOOK: type: ALTER_MATERIALIZED_VIEW_REBUILD
diff --git a/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_5.q.out b/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_5.q.out
index 2da384f4bbd..abeab03cf39 100644
--- a/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_5.q.out
+++ b/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_5.q.out
@@ -482,10 +482,16 @@ PREHOOK: query: UPDATE cmv_basetable_2_n3 SET a=2 WHERE a=1
PREHOOK: type: QUERY
PREHOOK: Input: default@cmv_basetable_2_n3
PREHOOK: Output: default@cmv_basetable_2_n3
+PREHOOK: Output: default@cmv_basetable_2_n3
POSTHOOK: query: UPDATE cmv_basetable_2_n3 SET a=2 WHERE a=1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@cmv_basetable_2_n3
POSTHOOK: Output: default@cmv_basetable_2_n3
+POSTHOOK: Output: default@cmv_basetable_2_n3
+POSTHOOK: Lineage: cmv_basetable_2_n3.a SIMPLE []
+POSTHOOK: Lineage: cmv_basetable_2_n3.b SIMPLE [(cmv_basetable_2_n3)cmv_basetable_2_n3.FieldSchema(name:b, type:varchar(256), comment:null), ]
+POSTHOOK: Lineage: cmv_basetable_2_n3.c SIMPLE [(cmv_basetable_2_n3)cmv_basetable_2_n3.FieldSchema(name:c, type:decimal(10,2), comment:null), ]
+POSTHOOK: Lineage: cmv_basetable_2_n3.d SIMPLE [(cmv_basetable_2_n3)cmv_basetable_2_n3.FieldSchema(name:d, type:int, comment:null), ]
PREHOOK: query: EXPLAIN
ALTER MATERIALIZED VIEW cmv_mat_view_n6 REBUILD
PREHOOK: type: ALTER_MATERIALIZED_VIEW_REBUILD
diff --git a/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_8.q.out b/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_8.q.out
index 218f70b8fff..93f519af220 100644
--- a/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_8.q.out
+++ b/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_8.q.out
@@ -352,10 +352,16 @@ PREHOOK: query: UPDATE cmv_basetable_n6 SET a=2 WHERE a=1
PREHOOK: type: QUERY
PREHOOK: Input: default@cmv_basetable_n6
PREHOOK: Output: default@cmv_basetable_n6
+PREHOOK: Output: default@cmv_basetable_n6
POSTHOOK: query: UPDATE cmv_basetable_n6 SET a=2 WHERE a=1
POSTHOOK: type: QUERY
POSTHOOK: Input: default@cmv_basetable_n6
POSTHOOK: Output: default@cmv_basetable_n6
+POSTHOOK: Output: default@cmv_basetable_n6
+POSTHOOK: Lineage: cmv_basetable_n6.a SIMPLE []
+POSTHOOK: Lineage: cmv_basetable_n6.b SIMPLE [(cmv_basetable_n6)cmv_basetable_n6.FieldSchema(name:b, type:varchar(256), comment:null), ]
+POSTHOOK: Lineage: cmv_basetable_n6.c SIMPLE [(cmv_basetable_n6)cmv_basetable_n6.FieldSchema(name:c, type:decimal(10,2), comment:null), ]
+POSTHOOK: Lineage: cmv_basetable_n6.d SIMPLE [(cmv_basetable_n6)cmv_basetable_n6.FieldSchema(name:d, type:int, comment:null), ]
PREHOOK: query: EXPLAIN
ALTER MATERIALIZED VIEW cmv_mat_view_n6 REBUILD
PREHOOK: type: ALTER_MATERIALIZED_VIEW_REBUILD
diff --git a/ql/src/test/results/clientpositive/llap/schema_evol_orc_acid_part_update.q.out b/ql/src/test/results/clientpositive/llap/schema_evol_orc_acid_part_update.q.out
index 67bafdbfced..82ffbdedc5b 100644
--- a/ql/src/test/results/clientpositive/llap/schema_evol_orc_acid_part_update.q.out
+++ b/ql/src/test/results/clientpositive/llap/schema_evol_orc_acid_part_update.q.out
@@ -123,6 +123,7 @@ PREHOOK: type: QUERY
PREHOOK: Input: default@partitioned_update_1_n1
PREHOOK: Input: default@partitioned_update_1_n1@part=1
PREHOOK: Input: default@partitioned_update_1_n1@part=2
+PREHOOK: Output: default@partitioned_update_1_n1
PREHOOK: Output: default@partitioned_update_1_n1@part=1
PREHOOK: Output: default@partitioned_update_1_n1@part=2
POSTHOOK: query: update partitioned_update_1_n1 set c=99
@@ -130,9 +131,22 @@ POSTHOOK: type: QUERY
POSTHOOK: Input: default@partitioned_update_1_n1
POSTHOOK: Input: default@partitioned_update_1_n1@part=1
POSTHOOK: Input: default@partitioned_update_1_n1@part=2
+POSTHOOK: Output: default@partitioned_update_1_n1
POSTHOOK: Output: default@partitioned_update_1_n1@part=1
+POSTHOOK: Output: default@partitioned_update_1_n1@part=1
+POSTHOOK: Output: default@partitioned_update_1_n1@part=2
POSTHOOK: Output: default@partitioned_update_1_n1@part=2
-row__id insert_num a b _c4 d part
+POSTHOOK: Lineage: partitioned_update_1_n1 PARTITION(part=1).a SIMPLE [(partitioned_update_1_n1)partitioned_update_1_n1.FieldSchema(name:a, type:int, comment:null), ]
+POSTHOOK: Lineage: partitioned_update_1_n1 PARTITION(part=1).b SIMPLE [(partitioned_update_1_n1)partitioned_update_1_n1.FieldSchema(name:b, type:string, comment:null), ]
+POSTHOOK: Lineage: partitioned_update_1_n1 PARTITION(part=1).c SIMPLE []
+POSTHOOK: Lineage: partitioned_update_1_n1 PARTITION(part=1).d SIMPLE [(partitioned_update_1_n1)partitioned_update_1_n1.FieldSchema(name:d, type:string, comment:null), ]
+POSTHOOK: Lineage: partitioned_update_1_n1 PARTITION(part=1).insert_num SIMPLE [(partitioned_update_1_n1)partitioned_update_1_n1.FieldSchema(name:insert_num, type:int, comment:null), ]
+POSTHOOK: Lineage: partitioned_update_1_n1 PARTITION(part=2).a SIMPLE [(partitioned_update_1_n1)partitioned_update_1_n1.FieldSchema(name:a, type:int, comment:null), ]
+POSTHOOK: Lineage: partitioned_update_1_n1 PARTITION(part=2).b SIMPLE [(partitioned_update_1_n1)partitioned_update_1_n1.FieldSchema(name:b, type:string, comment:null), ]
+POSTHOOK: Lineage: partitioned_update_1_n1 PARTITION(part=2).c SIMPLE []
+POSTHOOK: Lineage: partitioned_update_1_n1 PARTITION(part=2).d SIMPLE [(partitioned_update_1_n1)partitioned_update_1_n1.FieldSchema(name:d, type:string, comment:null), ]
+POSTHOOK: Lineage: partitioned_update_1_n1 PARTITION(part=2).insert_num SIMPLE [(partitioned_update_1_n1)partitioned_update_1_n1.FieldSchema(name:insert_num, type:int, comment:null), ]
+s.insert_num s.a s.b s.c s.d s.part
PREHOOK: query: select insert_num,part,a,b,c,d from partitioned_update_1_n1
PREHOOK: type: QUERY
PREHOOK: Input: default@partitioned_update_1_n1
diff --git a/ql/src/test/results/clientpositive/llap/schema_evol_orc_acid_part_update_llap_io.q.out b/ql/src/test/results/clientpositive/llap/schema_evol_orc_acid_part_update_llap_io.q.out
index 6817b78340b..5ec39113971 100644
--- a/ql/src/test/results/clientpositive/llap/schema_evol_orc_acid_part_update_llap_io.q.out
+++ b/ql/src/test/results/clientpositive/llap/schema_evol_orc_acid_part_update_llap_io.q.out
@@ -123,6 +123,7 @@ PREHOOK: type: QUERY
PREHOOK: Input: default@partitioned_update_1
PREHOOK: Input: default@partitioned_update_1@part=1
PREHOOK: Input: default@partitioned_update_1@part=2
+PREHOOK: Output: default@partitioned_update_1
PREHOOK: Output: default@partitioned_update_1@part=1
PREHOOK: Output: default@partitioned_update_1@part=2
POSTHOOK: query: update partitioned_update_1 set c=99
@@ -130,9 +131,22 @@ POSTHOOK: type: QUERY
POSTHOOK: Input: default@partitioned_update_1
POSTHOOK: Input: default@partitioned_update_1@part=1
POSTHOOK: Input: default@partitioned_update_1@part=2
+POSTHOOK: Output: default@partitioned_update_1
POSTHOOK: Output: default@partitioned_update_1@part=1
+POSTHOOK: Output: default@partitioned_update_1@part=1
+POSTHOOK: Output: default@partitioned_update_1@part=2
POSTHOOK: Output: default@partitioned_update_1@part=2
-row__id insert_num a b _c4 d part
+POSTHOOK: Lineage: partitioned_update_1 PARTITION(part=1).a SIMPLE [(partitioned_update_1)partitioned_update_1.FieldSchema(name:a, type:int, comment:null), ]
+POSTHOOK: Lineage: partitioned_update_1 PARTITION(part=1).b SIMPLE [(partitioned_update_1)partitioned_update_1.FieldSchema(name:b, type:string, comment:null), ]
+POSTHOOK: Lineage: partitioned_update_1 PARTITION(part=1).c SIMPLE []
+POSTHOOK: Lineage: partitioned_update_1 PARTITION(part=1).d SIMPLE [(partitioned_update_1)partitioned_update_1.FieldSchema(name:d, type:string, comment:null), ]
+POSTHOOK: Lineage: partitioned_update_1 PARTITION(part=1).insert_num SIMPLE [(partitioned_update_1)partitioned_update_1.FieldSchema(name:insert_num, type:int, comment:null), ]
+POSTHOOK: Lineage: partitioned_update_1 PARTITION(part=2).a SIMPLE [(partitioned_update_1)partitioned_update_1.FieldSchema(name:a, type:int, comment:null), ]
+POSTHOOK: Lineage: partitioned_update_1 PARTITION(part=2).b SIMPLE [(partitioned_update_1)partitioned_update_1.FieldSchema(name:b, type:string, comment:null), ]
+POSTHOOK: Lineage: partitioned_update_1 PARTITION(part=2).c SIMPLE []
+POSTHOOK: Lineage: partitioned_update_1 PARTITION(part=2).d SIMPLE [(partitioned_update_1)partitioned_update_1.FieldSchema(name:d, type:string, comment:null), ]
+POSTHOOK: Lineage: partitioned_update_1 PARTITION(part=2).insert_num SIMPLE [(partitioned_update_1)partitioned_update_1.FieldSchema(name:insert_num, type:int, comment:null), ]
+s.insert_num s.a s.b s.c s.d s.part
PREHOOK: query: select insert_num,part,a,b,c,d from partitioned_update_1
PREHOOK: type: QUERY
PREHOOK: Input: default@partitioned_update_1
diff --git a/ql/src/test/results/clientpositive/llap/schema_evol_orc_acid_table_update.q.out b/ql/src/test/results/clientpositive/llap/schema_evol_orc_acid_table_update.q.out
index 16c90820871..210302edd26 100644
--- a/ql/src/test/results/clientpositive/llap/schema_evol_orc_acid_table_update.q.out
+++ b/ql/src/test/results/clientpositive/llap/schema_evol_orc_acid_table_update.q.out
@@ -104,11 +104,18 @@ PREHOOK: query: update table5_n2 set c=99
PREHOOK: type: QUERY
PREHOOK: Input: default@table5_n2
PREHOOK: Output: default@table5_n2
+PREHOOK: Output: default@table5_n2
POSTHOOK: query: update table5_n2 set c=99
POSTHOOK: type: QUERY
POSTHOOK: Input: default@table5_n2
POSTHOOK: Output: default@table5_n2
-row__id insert_num a b _c4 d
+POSTHOOK: Output: default@table5_n2
+POSTHOOK: Lineage: table5_n2.a SIMPLE [(table5_n2)table5_n2.FieldSchema(name:a, type:int, comment:null), ]
+POSTHOOK: Lineage: table5_n2.b SIMPLE [(table5_n2)table5_n2.FieldSchema(name:b, type:string, comment:null), ]
+POSTHOOK: Lineage: table5_n2.c SIMPLE []
+POSTHOOK: Lineage: table5_n2.d SIMPLE [(table5_n2)table5_n2.FieldSchema(name:d, type:string, comment:null), ]
+POSTHOOK: Lineage: table5_n2.insert_num SIMPLE [(table5_n2)table5_n2.FieldSchema(name:insert_num, type:int, comment:null), ]
+s.insert_num s.a s.b s.c s.d
PREHOOK: query: select a,b,c,d from table5_n2
PREHOOK: type: QUERY
PREHOOK: Input: default@table5_n2
diff --git a/ql/src/test/results/clientpositive/llap/schema_evol_orc_acid_table_update_llap_io.q.out b/ql/src/test/results/clientpositive/llap/schema_evol_orc_acid_table_update_llap_io.q.out
index 79aa17fdde0..dc7cad88c12 100644
--- a/ql/src/test/results/clientpositive/llap/schema_evol_orc_acid_table_update_llap_io.q.out
+++ b/ql/src/test/results/clientpositive/llap/schema_evol_orc_acid_table_update_llap_io.q.out
@@ -104,11 +104,18 @@ PREHOOK: query: update table5_n0 set c=99
PREHOOK: type: QUERY
PREHOOK: Input: default@table5_n0
PREHOOK: Output: default@table5_n0
+PREHOOK: Output: default@table5_n0
POSTHOOK: query: update table5_n0 set c=99
POSTHOOK: type: QUERY
POSTHOOK: Input: default@table5_n0
POSTHOOK: Output: default@table5_n0
-row__id insert_num a b _c4 d
+POSTHOOK: Output: default@table5_n0
+POSTHOOK: Lineage: table5_n0.a SIMPLE [(table5_n0)table5_n0.FieldSchema(name:a, type:int, comment:null), ]
+POSTHOOK: Lineage: table5_n0.b SIMPLE [(table5_n0)table5_n0.FieldSchema(name:b, type:string, comment:null), ]
+POSTHOOK: Lineage: table5_n0.c SIMPLE []
+POSTHOOK: Lineage: table5_n0.d SIMPLE [(table5_n0)table5_n0.FieldSchema(name:d, type:string, comment:null), ]
+POSTHOOK: Lineage: table5_n0.insert_num SIMPLE [(table5_n0)table5_n0.FieldSchema(name:insert_num, type:int, comment:null), ]
+s.insert_num s.a s.b s.c s.d
PREHOOK: query: select a,b,c,d from table5_n0
PREHOOK: type: QUERY
PREHOOK: Input: default@table5_n0
diff --git a/ql/src/test/results/clientpositive/llap/schema_evol_orc_acidvec_part_update.q.out b/ql/src/test/results/clientpositive/llap/schema_evol_orc_acidvec_part_update.q.out
index cdf2ee2fbd3..e0c60770fb7 100644
--- a/ql/src/test/results/clientpositive/llap/schema_evol_orc_acidvec_part_update.q.out
+++ b/ql/src/test/results/clientpositive/llap/schema_evol_orc_acidvec_part_update.q.out
@@ -123,6 +123,7 @@ PREHOOK: type: QUERY
PREHOOK: Input: default@partitioned_update_1_n2
PREHOOK: Input: default@partitioned_update_1_n2@part=1
PREHOOK: Input: default@partitioned_update_1_n2@part=2
+PREHOOK: Output: default@partitioned_update_1_n2
PREHOOK: Output: default@partitioned_update_1_n2@part=1
PREHOOK: Output: default@partitioned_update_1_n2@part=2
POSTHOOK: query: update partitioned_update_1_n2 set c=99
@@ -130,9 +131,22 @@ POSTHOOK: type: QUERY
POSTHOOK: Input: default@partitioned_update_1_n2
POSTHOOK: Input: default@partitioned_update_1_n2@part=1
POSTHOOK: Input: default@partitioned_update_1_n2@part=2
+POSTHOOK: Output: default@partitioned_update_1_n2
POSTHOOK: Output: default@partitioned_update_1_n2@part=1
+POSTHOOK: Output: default@partitioned_update_1_n2@part=1
+POSTHOOK: Output: default@partitioned_update_1_n2@part=2
POSTHOOK: Output: default@partitioned_update_1_n2@part=2
-row__id insert_num a b _c4 d part
+POSTHOOK: Lineage: partitioned_update_1_n2 PARTITION(part=1).a SIMPLE [(partitioned_update_1_n2)partitioned_update_1_n2.FieldSchema(name:a, type:int, comment:null), ]
+POSTHOOK: Lineage: partitioned_update_1_n2 PARTITION(part=1).b SIMPLE [(partitioned_update_1_n2)partitioned_update_1_n2.FieldSchema(name:b, type:string, comment:null), ]
+POSTHOOK: Lineage: partitioned_update_1_n2 PARTITION(part=1).c SIMPLE []
+POSTHOOK: Lineage: partitioned_update_1_n2 PARTITION(part=1).d SIMPLE [(partitioned_update_1_n2)partitioned_update_1_n2.FieldSchema(name:d, type:string, comment:null), ]
+POSTHOOK: Lineage: partitioned_update_1_n2 PARTITION(part=1).insert_num SIMPLE [(partitioned_update_1_n2)partitioned_update_1_n2.FieldSchema(name:insert_num, type:int, comment:null), ]
+POSTHOOK: Lineage: partitioned_update_1_n2 PARTITION(part=2).a SIMPLE [(partitioned_update_1_n2)partitioned_update_1_n2.FieldSchema(name:a, type:int, comment:null), ]
+POSTHOOK: Lineage: partitioned_update_1_n2 PARTITION(part=2).b SIMPLE [(partitioned_update_1_n2)partitioned_update_1_n2.FieldSchema(name:b, type:string, comment:null), ]
+POSTHOOK: Lineage: partitioned_update_1_n2 PARTITION(part=2).c SIMPLE []
+POSTHOOK: Lineage: partitioned_update_1_n2 PARTITION(part=2).d SIMPLE [(partitioned_update_1_n2)partitioned_update_1_n2.FieldSchema(name:d, type:string, comment:null), ]
+POSTHOOK: Lineage: partitioned_update_1_n2 PARTITION(part=2).insert_num SIMPLE [(partitioned_update_1_n2)partitioned_update_1_n2.FieldSchema(name:insert_num, type:int, comment:null), ]
+s.insert_num s.a s.b s.c s.d s.part
PREHOOK: query: select insert_num,part,a,b,c,d from partitioned_update_1_n2
PREHOOK: type: QUERY
PREHOOK: Input: default@partitioned_update_1_n2
diff --git a/ql/src/test/results/clientpositive/llap/schema_evol_orc_acidvec_part_update_llap_io.q.out b/ql/src/test/results/clientpositive/llap/schema_evol_orc_acidvec_part_update_llap_io.q.out
index 556c559c66e..121066db980 100644
--- a/ql/src/test/results/clientpositive/llap/schema_evol_orc_acidvec_part_update_llap_io.q.out
+++ b/ql/src/test/results/clientpositive/llap/schema_evol_orc_acidvec_part_update_llap_io.q.out
@@ -123,6 +123,7 @@ PREHOOK: type: QUERY
PREHOOK: Input: default@partitioned_update_1_n0
PREHOOK: Input: default@partitioned_update_1_n0@part=1
PREHOOK: Input: default@partitioned_update_1_n0@part=2
+PREHOOK: Output: default@partitioned_update_1_n0
PREHOOK: Output: default@partitioned_update_1_n0@part=1
PREHOOK: Output: default@partitioned_update_1_n0@part=2
POSTHOOK: query: update partitioned_update_1_n0 set c=99
@@ -130,9 +131,22 @@ POSTHOOK: type: QUERY
POSTHOOK: Input: default@partitioned_update_1_n0
POSTHOOK: Input: default@partitioned_update_1_n0@part=1
POSTHOOK: Input: default@partitioned_update_1_n0@part=2
+POSTHOOK: Output: default@partitioned_update_1_n0
POSTHOOK: Output: default@partitioned_update_1_n0@part=1
+POSTHOOK: Output: default@partitioned_update_1_n0@part=1
+POSTHOOK: Output: default@partitioned_update_1_n0@part=2
POSTHOOK: Output: default@partitioned_update_1_n0@part=2
-row__id insert_num a b _c4 d part
+POSTHOOK: Lineage: partitioned_update_1_n0 PARTITION(part=1).a SIMPLE [(partitioned_update_1_n0)partitioned_update_1_n0.FieldSchema(name:a, type:int, comment:null), ]
+POSTHOOK: Lineage: partitioned_update_1_n0 PARTITION(part=1).b SIMPLE [(partitioned_update_1_n0)partitioned_update_1_n0.FieldSchema(name:b, type:string, comment:null), ]
+POSTHOOK: Lineage: partitioned_update_1_n0 PARTITION(part=1).c SIMPLE []
+POSTHOOK: Lineage: partitioned_update_1_n0 PARTITION(part=1).d SIMPLE [(partitioned_update_1_n0)partitioned_update_1_n0.FieldSchema(name:d, type:string, comment:null), ]
+POSTHOOK: Lineage: partitioned_update_1_n0 PARTITION(part=1).insert_num SIMPLE [(partitioned_update_1_n0)partitioned_update_1_n0.FieldSchema(name:insert_num, type:int, comment:null), ]
+POSTHOOK: Lineage: partitioned_update_1_n0 PARTITION(part=2).a SIMPLE [(partitioned_update_1_n0)partitioned_update_1_n0.FieldSchema(name:a, type:int, comment:null), ]
+POSTHOOK: Lineage: partitioned_update_1_n0 PARTITION(part=2).b SIMPLE [(partitioned_update_1_n0)partitioned_update_1_n0.FieldSchema(name:b, type:string, comment:null), ]
+POSTHOOK: Lineage: partitioned_update_1_n0 PARTITION(part=2).c SIMPLE []
+POSTHOOK: Lineage: partitioned_update_1_n0 PARTITION(part=2).d SIMPLE [(partitioned_update_1_n0)partitioned_update_1_n0.FieldSchema(name:d, type:string, comment:null), ]
+POSTHOOK: Lineage: partitioned_update_1_n0 PARTITION(part=2).insert_num SIMPLE [(partitioned_update_1_n0)partitioned_update_1_n0.FieldSchema(name:insert_num, type:int, comment:null), ]
+s.insert_num s.a s.b s.c s.d s.part
PREHOOK: query: select insert_num,part,a,b,c,d from partitioned_update_1_n0
PREHOOK: type: QUERY
PREHOOK: Input: default@partitioned_update_1_n0
diff --git a/ql/src/test/results/clientpositive/llap/schema_evol_orc_acidvec_table_update.q.out b/ql/src/test/results/clientpositive/llap/schema_evol_orc_acidvec_table_update.q.out
index 7e314538155..854c256df61 100644
--- a/ql/src/test/results/clientpositive/llap/schema_evol_orc_acidvec_table_update.q.out
+++ b/ql/src/test/results/clientpositive/llap/schema_evol_orc_acidvec_table_update.q.out
@@ -104,11 +104,18 @@ PREHOOK: query: update table5_n3 set c=99
PREHOOK: type: QUERY
PREHOOK: Input: default@table5_n3
PREHOOK: Output: default@table5_n3
+PREHOOK: Output: default@table5_n3
POSTHOOK: query: update table5_n3 set c=99
POSTHOOK: type: QUERY
POSTHOOK: Input: default@table5_n3
POSTHOOK: Output: default@table5_n3
-row__id insert_num a b _c4 d
+POSTHOOK: Output: default@table5_n3
+POSTHOOK: Lineage: table5_n3.a SIMPLE [(table5_n3)table5_n3.FieldSchema(name:a, type:int, comment:null), ]
+POSTHOOK: Lineage: table5_n3.b SIMPLE [(table5_n3)table5_n3.FieldSchema(name:b, type:string, comment:null), ]
+POSTHOOK: Lineage: table5_n3.c SIMPLE []
+POSTHOOK: Lineage: table5_n3.d SIMPLE [(table5_n3)table5_n3.FieldSchema(name:d, type:string, comment:null), ]
+POSTHOOK: Lineage: table5_n3.insert_num SIMPLE [(table5_n3)table5_n3.FieldSchema(name:insert_num, type:int, comment:null), ]
+s.insert_num s.a s.b s.c s.d
PREHOOK: query: select a,b,c,d from table5_n3
PREHOOK: type: QUERY
PREHOOK: Input: default@table5_n3
diff --git a/ql/src/test/results/clientpositive/llap/schema_evol_orc_acidvec_table_update_llap_io.q.out b/ql/src/test/results/clientpositive/llap/schema_evol_orc_acidvec_table_update_llap_io.q.out
index 06db5da4fe9..536f41e0fa5 100644
--- a/ql/src/test/results/clientpositive/llap/schema_evol_orc_acidvec_table_update_llap_io.q.out
+++ b/ql/src/test/results/clientpositive/llap/schema_evol_orc_acidvec_table_update_llap_io.q.out
@@ -104,11 +104,18 @@ PREHOOK: query: update table5 set c=99
PREHOOK: type: QUERY
PREHOOK: Input: default@table5
PREHOOK: Output: default@table5
+PREHOOK: Output: default@table5
POSTHOOK: query: update table5 set c=99
POSTHOOK: type: QUERY
POSTHOOK: Input: default@table5
POSTHOOK: Output: default@table5
-row__id insert_num a b _c4 d
+POSTHOOK: Output: default@table5
+POSTHOOK: Lineage: table5.a SIMPLE [(table5)table5.FieldSchema(name:a, type:int, comment:null), ]
+POSTHOOK: Lineage: table5.b SIMPLE [(table5)table5.FieldSchema(name:b, type:string, comment:null), ]
+POSTHOOK: Lineage: table5.c SIMPLE []
+POSTHOOK: Lineage: table5.d SIMPLE [(table5)table5.FieldSchema(name:d, type:string, comment:null), ]
+POSTHOOK: Lineage: table5.insert_num SIMPLE [(table5)table5.FieldSchema(name:insert_num, type:int, comment:null), ]
+s.insert_num s.a s.b s.c s.d
PREHOOK: query: select a,b,c,d from table5
PREHOOK: type: QUERY
PREHOOK: Input: default@table5
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 a00bac9d666..ae7b2ce2741 100644
--- a/ql/src/test/results/clientpositive/llap/sort_acid.q.out
+++ b/ql/src/test/results/clientpositive/llap/sort_acid.q.out
@@ -64,24 +64,29 @@ update acidtlb set b=777
PREHOOK: type: QUERY
PREHOOK: Input: default@acidtlb
PREHOOK: Output: default@acidtlb
+PREHOOK: Output: default@acidtlb
POSTHOOK: query: explain cbo
update acidtlb set b=777
POSTHOOK: type: QUERY
POSTHOOK: Input: default@acidtlb
POSTHOOK: Output: default@acidtlb
+POSTHOOK: Output: default@acidtlb
CBO PLAN:
-HiveSortExchange(distribution=[any], collation=[[0]])
- HiveProject(row__id=[$4], a=[$0], _o__c2=[777])
- HiveTableScan(table=[[default, acidtlb]], table:alias=[acidtlb])
+HiveProject(row__id=[$4], a=[$0], b=[777])
+ HiveTableScan(table=[[default, acidtlb]], table:alias=[acidtlb])
PREHOOK: query: update acidtlb set b=777
PREHOOK: type: QUERY
PREHOOK: Input: default@acidtlb
PREHOOK: Output: default@acidtlb
+PREHOOK: Output: default@acidtlb
POSTHOOK: query: update acidtlb set b=777
POSTHOOK: type: QUERY
POSTHOOK: Input: default@acidtlb
POSTHOOK: Output: default@acidtlb
+POSTHOOK: Output: default@acidtlb
+POSTHOOK: Lineage: acidtlb.a SIMPLE [(acidtlb)acidtlb.FieldSchema(name:a, type:int, comment:null), ]
+POSTHOOK: Lineage: acidtlb.b SIMPLE []
PREHOOK: query: select * from acidtlb
PREHOOK: type: QUERY
PREHOOK: Input: default@acidtlb
@@ -92,43 +97,48 @@ POSTHOOK: Input: default@acidtlb
#### A masked pattern was here ####
10 777
30 777
-Warning: Shuffle Join MERGEJOIN[22][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+Warning: Shuffle Join MERGEJOIN[27][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
PREHOOK: query: explain cbo
update acidtlb set b=350
where a in (select a from acidtlb where a = 30)
PREHOOK: type: QUERY
PREHOOK: Input: default@acidtlb
PREHOOK: Output: default@acidtlb
+PREHOOK: Output: default@acidtlb
POSTHOOK: query: explain cbo
update acidtlb set b=350
where a in (select a from acidtlb where a = 30)
POSTHOOK: type: QUERY
POSTHOOK: Input: default@acidtlb
POSTHOOK: Output: default@acidtlb
+POSTHOOK: Output: default@acidtlb
CBO PLAN:
-HiveSortExchange(distribution=[any], collation=[[0]])
- HiveProject(row__id=[$1], a=[CAST(30):INTEGER], _o__c2=[350])
- HiveSemiJoin(condition=[true], joinType=[semi])
- HiveProject(a=[CAST(30):INTEGER], ROW__ID=[$4])
- HiveFilter(condition=[=($0, 30)])
- HiveTableScan(table=[[default, acidtlb]], table:alias=[acidtlb])
- HiveProject($f0=[$0])
- HiveAggregate(group=[{0}])
- HiveProject($f0=[true])
- HiveFilter(condition=[=($0, 30)])
- HiveTableScan(table=[[default, acidtlb]], table:alias=[acidtlb])
+HiveProject(row__id=[$1], a=[CAST(30):INTEGER], b=[350])
+ HiveSemiJoin(condition=[true], joinType=[semi])
+ HiveProject(a=[CAST(30):INTEGER], ROW__ID=[$4])
+ HiveFilter(condition=[=($0, 30)])
+ HiveTableScan(table=[[default, acidtlb]], table:alias=[acidtlb])
+ HiveProject($f0=[$0])
+ HiveAggregate(group=[{0}])
+ HiveProject($f0=[true])
+ HiveFilter(condition=[=($0, 30)])
+ HiveTableScan(table=[[default, acidtlb]], table:alias=[acidtlb])
-Warning: Shuffle Join MERGEJOIN[22][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+Warning: Shuffle Join MERGEJOIN[27][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
PREHOOK: query: update acidtlb set b=350
where a in (select a from acidtlb where a = 30)
PREHOOK: type: QUERY
PREHOOK: Input: default@acidtlb
PREHOOK: Output: default@acidtlb
+PREHOOK: Output: default@acidtlb
POSTHOOK: query: update acidtlb set b=350
where a in (select a from acidtlb where a = 30)
POSTHOOK: type: QUERY
POSTHOOK: Input: default@acidtlb
POSTHOOK: Output: default@acidtlb
+POSTHOOK: Output: default@acidtlb
+POSTHOOK: Lineage: acidtlb.a SIMPLE []
+POSTHOOK: Lineage: acidtlb.b SIMPLE []
PREHOOK: query: select * from acidtlb
PREHOOK: type: QUERY
PREHOOK: Input: default@acidtlb
@@ -146,6 +156,7 @@ PREHOOK: type: QUERY
PREHOOK: Input: default@acidtlb
PREHOOK: Input: default@othertlb
PREHOOK: Output: default@acidtlb
+PREHOOK: Output: default@acidtlb
POSTHOOK: query: explain cbo
update acidtlb set b=450
where a in (select c from othertlb where c < 65)
@@ -153,16 +164,16 @@ POSTHOOK: type: QUERY
POSTHOOK: Input: default@acidtlb
POSTHOOK: Input: default@othertlb
POSTHOOK: Output: default@acidtlb
+POSTHOOK: Output: default@acidtlb
CBO PLAN:
-HiveSortExchange(distribution=[any], collation=[[0]])
- HiveProject(row__id=[$1], a=[$0], _o__c2=[450])
- HiveSemiJoin(condition=[=($0, $2)], joinType=[semi])
- HiveProject(a=[$0], ROW__ID=[$4])
- HiveFilter(condition=[<($0, 65)])
- HiveTableScan(table=[[default, acidtlb]], table:alias=[acidtlb])
- HiveProject(c=[$0])
- HiveFilter(condition=[<($0, 65)])
- HiveTableScan(table=[[default, othertlb]], table:alias=[othertlb])
+HiveProject(row__id=[$1], a=[$0], b=[450])
+ HiveSemiJoin(condition=[=($0, $2)], joinType=[semi])
+ HiveProject(a=[$0], ROW__ID=[$4])
+ HiveFilter(condition=[<($0, 65)])
+ HiveTableScan(table=[[default, acidtlb]], table:alias=[acidtlb])
+ HiveProject(c=[$0])
+ HiveFilter(condition=[<($0, 65)])
+ HiveTableScan(table=[[default, othertlb]], table:alias=[othertlb])
PREHOOK: query: update acidtlb set b=450
where a in (select c from othertlb where c < 65)
@@ -170,12 +181,16 @@ PREHOOK: type: QUERY
PREHOOK: Input: default@acidtlb
PREHOOK: Input: default@othertlb
PREHOOK: Output: default@acidtlb
+PREHOOK: Output: default@acidtlb
POSTHOOK: query: update acidtlb set b=450
where a in (select c from othertlb where c < 65)
POSTHOOK: type: QUERY
POSTHOOK: Input: default@acidtlb
POSTHOOK: Input: default@othertlb
POSTHOOK: Output: default@acidtlb
+POSTHOOK: Output: default@acidtlb
+POSTHOOK: Lineage: acidtlb.a SIMPLE [(acidtlb)acidtlb.FieldSchema(name:a, type:int, comment:null), ]
+POSTHOOK: Lineage: acidtlb.b SIMPLE []
PREHOOK: query: select * from acidtlb
PREHOOK: type: QUERY
PREHOOK: Input: default@acidtlb
diff --git a/ql/src/test/results/clientpositive/llap/split_update.q.out b/ql/src/test/results/clientpositive/llap/split_update.q.out
new file mode 100644
index 00000000000..fa74d01a073
--- /dev/null
+++ b/ql/src/test/results/clientpositive/llap/split_update.q.out
@@ -0,0 +1,225 @@
+PREHOOK: query: create table acid_uami_n0(i int,
+ de decimal(5,2) constraint nn1 not null enforced,
+ vc varchar(128) constraint ch2 CHECK (de >= cast(i as decimal(5,2))) enforced)
+ clustered by (i) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@acid_uami_n0
+POSTHOOK: query: create table acid_uami_n0(i int,
+ de decimal(5,2) constraint nn1 not null enforced,
+ vc varchar(128) constraint ch2 CHECK (de >= cast(i as decimal(5,2))) enforced)
+ clustered by (i) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@acid_uami_n0
+PREHOOK: query: explain update acid_uami_n0 set de = 893.14 where de = 103.00 or de = 119.00
+PREHOOK: type: QUERY
+PREHOOK: Input: default@acid_uami_n0
+PREHOOK: Output: default@acid_uami_n0
+PREHOOK: Output: default@acid_uami_n0
+POSTHOOK: query: explain update acid_uami_n0 set de = 893.14 where de = 103.00 or de = 119.00
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@acid_uami_n0
+POSTHOOK: Output: default@acid_uami_n0
+POSTHOOK: Output: default@acid_uami_n0
+STAGE DEPENDENCIES:
+ Stage-2 is a root stage
+ Stage-3 depends on stages: Stage-2
+ Stage-0 depends on stages: Stage-3
+ Stage-4 depends on stages: Stage-0
+ Stage-1 depends on stages: Stage-3
+ Stage-5 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-2
+ Tez
+#### A masked pattern was here ####
+ Edges:
+ Reducer 2 <- Map 1 (SIMPLE_EDGE)
+ Reducer 3 <- Map 1 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: acid_uami_n0
+ filterExpr: (de) IN (103, 119) (type: boolean)
+ Statistics: Num rows: 1 Data size: 328 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (de) IN (103, 119) (type: boolean)
+ Statistics: Num rows: 1 Data size: 328 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), i (type: int), vc (type: varchar(128))
+ outputColumnNames: _col0, _col1, _col3
+ Statistics: Num rows: 1 Data size: 328 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 328 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)
+ Statistics: Num rows: 1 Data size: 328 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col1 (type: int), 893.14 (type: decimal(5,2)), _col3 (type: varchar(128))
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 1 Data size: 328 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: enforce_constraint((_col1 is not null and (893.14 >= CAST( _col0 AS decimal(5,2))) is not false)) (type: boolean)
+ Statistics: Num rows: 1 Data size: 328 Basic stats: COMPLETE Column stats: NONE
+ 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: 328 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col1 (type: decimal(5,2)), _col2 (type: varchar(128))
+ Execution mode: vectorized, llap
+ LLAP IO: may be used (ACID table)
+ Reducer 2
+ Execution mode: vectorized, llap
+ Reduce Operator Tree:
+ Select Operator
+ expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 328 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 328 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.acid_uami_n0
+ Write Type: DELETE
+ Reducer 3
+ Execution mode: vectorized, llap
+ Reduce Operator Tree:
+ Select Operator
+ expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: decimal(5,2)), VALUE._col1 (type: varchar(128))
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 1 Data size: 328 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 328 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.acid_uami_n0
+ Write Type: INSERT
+
+ Stage: Stage-3
+ Dependency Collection
+
+ Stage: Stage-0
+ Move Operator
+ tables:
+ replace: false
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.acid_uami_n0
+ Write Type: DELETE
+
+ Stage: Stage-4
+ Stats Work
+ Basic Stats Work:
+
+ Stage: Stage-1
+ Move Operator
+ tables:
+ replace: false
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.acid_uami_n0
+ Write Type: INSERT
+
+ Stage: Stage-5
+ Stats Work
+ Basic Stats Work:
+
+PREHOOK: query: explain update acid_uami_n0 set de = 893.14 where de = 103.00 or de = 119.00
+PREHOOK: type: QUERY
+PREHOOK: Input: default@acid_uami_n0
+PREHOOK: Output: default@acid_uami_n0
+POSTHOOK: query: explain update acid_uami_n0 set de = 893.14 where de = 103.00 or de = 119.00
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@acid_uami_n0
+POSTHOOK: Output: default@acid_uami_n0
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-2 depends on stages: Stage-1
+ Stage-0 depends on stages: Stage-2
+ Stage-3 depends on stages: Stage-0
+
+STAGE PLANS:
+ Stage: Stage-1
+ Tez
+#### A masked pattern was here ####
+ Edges:
+ Reducer 2 <- Map 1 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: acid_uami_n0
+ filterExpr: ((de) IN (103, 119) and enforce_constraint((CAST( i AS decimal(5,2)) <= 893.14) is not false)) (type: boolean)
+ Statistics: Num rows: 1 Data size: 328 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: ((de) IN (103, 119) and enforce_constraint((CAST( i AS decimal(5,2)) <= 893.14) is not false)) (type: boolean)
+ Statistics: Num rows: 1 Data size: 328 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), i (type: int), vc (type: varchar(128))
+ outputColumnNames: _col0, _col1, _col3
+ Statistics: Num rows: 1 Data size: 328 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)
+ Statistics: Num rows: 1 Data size: 328 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col1 (type: int), 893.14 (type: decimal(5,2)), _col3 (type: varchar(128))
+ Execution mode: vectorized, llap
+ LLAP IO: may be used (ACID table)
+ Reducer 2
+ Execution mode: vectorized, llap
+ Reduce Operator Tree:
+ Select Operator
+ expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), VALUE._col0 (type: int), VALUE._col1 (type: decimal(5,2)), VALUE._col2 (type: varchar(128))
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Statistics: Num rows: 1 Data size: 328 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 328 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.acid_uami_n0
+ Write Type: UPDATE
+
+ Stage: Stage-2
+ Dependency Collection
+
+ Stage: Stage-0
+ Move Operator
+ tables:
+ replace: false
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.acid_uami_n0
+ Write Type: UPDATE
+
+ Stage: Stage-3
+ Stats Work
+ Basic Stats Work:
+
diff --git a/ql/src/test/results/clientpositive/llap/stats_part2.q.out b/ql/src/test/results/clientpositive/llap/stats_part2.q.out
index 57b0819e20e..e90f93a03c5 100644
--- a/ql/src/test/results/clientpositive/llap/stats_part2.q.out
+++ b/ql/src/test/results/clientpositive/llap/stats_part2.q.out
@@ -612,6 +612,7 @@ PREHOOK: type: QUERY
PREHOOK: Input: default@stats_part
PREHOOK: Input: default@stats_part@p=101
PREHOOK: Input: default@stats_part@p=102
+PREHOOK: Output: default@stats_part
PREHOOK: Output: default@stats_part@p=101
PREHOOK: Output: default@stats_part@p=102
POSTHOOK: query: update stats_part set key = key + 100 where key in(-50,40) and p > 100
@@ -619,8 +620,12 @@ POSTHOOK: type: QUERY
POSTHOOK: Input: default@stats_part
POSTHOOK: Input: default@stats_part@p=101
POSTHOOK: Input: default@stats_part@p=102
+POSTHOOK: Output: default@stats_part
+POSTHOOK: Output: default@stats_part@p=101
POSTHOOK: Output: default@stats_part@p=101
POSTHOOK: Output: default@stats_part@p=102
+POSTHOOK: Lineage: stats_part PARTITION(p=101).key EXPRESSION [(stats_part)stats_part.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: stats_part PARTITION(p=101).value SIMPLE [(stats_part)stats_part.FieldSchema(name:value, type:string, comment:null), ]
PREHOOK: query: explain select max(key) from stats_part where p > 100
PREHOOK: type: QUERY
PREHOOK: Input: default@stats_part
@@ -762,7 +767,7 @@ Partition Parameters:
numFiles 4
numRows 2
rawDataSize 0
- totalSize 2913
+ totalSize 2919
#### A masked pattern was here ####
# Storage Information
@@ -828,6 +833,7 @@ PREHOOK: type: QUERY
PREHOOK: Input: default@stats_part
PREHOOK: Input: default@stats_part@p=101
PREHOOK: Input: default@stats_part@p=102
+PREHOOK: Output: default@stats_part
PREHOOK: Output: default@stats_part@p=101
PREHOOK: Output: default@stats_part@p=102
POSTHOOK: query: update stats_part set value = concat(value, 'updated') where cast(key as integer) in(40,53) and p > 100
@@ -835,6 +841,7 @@ POSTHOOK: type: QUERY
POSTHOOK: Input: default@stats_part
POSTHOOK: Input: default@stats_part@p=101
POSTHOOK: Input: default@stats_part@p=102
+POSTHOOK: Output: default@stats_part
POSTHOOK: Output: default@stats_part@p=101
POSTHOOK: Output: default@stats_part@p=102
PREHOOK: query: desc formatted stats_part partition(p = 100)
@@ -896,7 +903,7 @@ Partition Parameters:
numFiles 4
numRows 2
rawDataSize 0
- totalSize 2913
+ totalSize 2919
#### A masked pattern was here ####
# Storage Information
@@ -1034,7 +1041,7 @@ Partition Parameters:
numFiles 5
numRows 1
rawDataSize 0
- totalSize 3611
+ totalSize 3617
#### A masked pattern was here ####
# Storage Information
diff --git a/ql/src/test/results/clientpositive/llap/update_after_multiple_inserts.q.out b/ql/src/test/results/clientpositive/llap/update_after_multiple_inserts.q.out
index 98a26e0e505..61e90fc68cf 100644
--- a/ql/src/test/results/clientpositive/llap/update_after_multiple_inserts.q.out
+++ b/ql/src/test/results/clientpositive/llap/update_after_multiple_inserts.q.out
@@ -56,10 +56,15 @@ PREHOOK: query: update acid_uami set de = 3.14 where de = 109.23 or de = 119.23
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_uami
PREHOOK: Output: default@acid_uami
+PREHOOK: Output: default@acid_uami
POSTHOOK: query: update acid_uami set de = 3.14 where de = 109.23 or de = 119.23
POSTHOOK: type: QUERY
POSTHOOK: Input: default@acid_uami
POSTHOOK: Output: default@acid_uami
+POSTHOOK: Output: default@acid_uami
+POSTHOOK: Lineage: acid_uami.de EXPRESSION []
+POSTHOOK: Lineage: acid_uami.i SIMPLE [(acid_uami)acid_uami.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: acid_uami.vc SIMPLE [(acid_uami)acid_uami.FieldSchema(name:vc, type:varchar(128), comment:null), ]
PREHOOK: query: select * from acid_uami order by de
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_uami
diff --git a/ql/src/test/results/clientpositive/llap/update_after_multiple_inserts_special_characters.q.out b/ql/src/test/results/clientpositive/llap/update_after_multiple_inserts_special_characters.q.out
index 07240930661..00bf33f52ae 100644
--- a/ql/src/test/results/clientpositive/llap/update_after_multiple_inserts_special_characters.q.out
+++ b/ql/src/test/results/clientpositive/llap/update_after_multiple_inserts_special_characters.q.out
@@ -56,10 +56,15 @@ PREHOOK: query: update `aci/d_u/ami` set `d?*de e` = 3.14 where `d?*de e` = 109.
PREHOOK: type: QUERY
PREHOOK: Input: default@aci/d_u/ami
PREHOOK: Output: default@aci/d_u/ami
+PREHOOK: Output: default@aci/d_u/ami
POSTHOOK: query: update `aci/d_u/ami` set `d?*de e` = 3.14 where `d?*de e` = 109.23 or `d?*de e` = 119.23
POSTHOOK: type: QUERY
POSTHOOK: Input: default@aci/d_u/ami
POSTHOOK: Output: default@aci/d_u/ami
+POSTHOOK: Output: default@aci/d_u/ami
+POSTHOOK: Lineage: aci/d_u/ami.d?*de e EXPRESSION []
+POSTHOOK: Lineage: aci/d_u/ami.i SIMPLE [(aci/d_u/ami)aci/d_u/ami.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: aci/d_u/ami.vc SIMPLE [(aci/d_u/ami)aci/d_u/ami.FieldSchema(name:vc, type:varchar(128), comment:null), ]
PREHOOK: query: select * from `aci/d_u/ami` order by `d?*de e`
PREHOOK: type: QUERY
PREHOOK: Input: default@aci/d_u/ami
diff --git a/ql/src/test/results/clientpositive/llap/update_all_non_partitioned.q.out b/ql/src/test/results/clientpositive/llap/update_all_non_partitioned.q.out
index 39dd71bd6fb..0c675280dcb 100644
--- a/ql/src/test/results/clientpositive/llap/update_all_non_partitioned.q.out
+++ b/ql/src/test/results/clientpositive/llap/update_all_non_partitioned.q.out
@@ -38,10 +38,14 @@ PREHOOK: query: update acid_uanp set b = 'fred'
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_uanp
PREHOOK: Output: default@acid_uanp
+PREHOOK: Output: default@acid_uanp
POSTHOOK: query: update acid_uanp set b = 'fred'
POSTHOOK: type: QUERY
POSTHOOK: Input: default@acid_uanp
POSTHOOK: Output: default@acid_uanp
+POSTHOOK: Output: default@acid_uanp
+POSTHOOK: Lineage: acid_uanp.a SIMPLE [(acid_uanp)acid_uanp.FieldSchema(name:a, type:int, comment:null), ]
+POSTHOOK: Lineage: acid_uanp.b EXPRESSION []
PREHOOK: query: select a,b from acid_uanp order by a
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_uanp
diff --git a/ql/src/test/results/clientpositive/llap/update_all_partitioned.q.out b/ql/src/test/results/clientpositive/llap/update_all_partitioned.q.out
index 78c02ceea2b..80d27d1adab 100644
--- a/ql/src/test/results/clientpositive/llap/update_all_partitioned.q.out
+++ b/ql/src/test/results/clientpositive/llap/update_all_partitioned.q.out
@@ -63,6 +63,7 @@ PREHOOK: type: QUERY
PREHOOK: Input: default@acid_uap
PREHOOK: Input: default@acid_uap@ds=today
PREHOOK: Input: default@acid_uap@ds=tomorrow
+PREHOOK: Output: default@acid_uap
PREHOOK: Output: default@acid_uap@ds=today
PREHOOK: Output: default@acid_uap@ds=tomorrow
POSTHOOK: query: update acid_uap set b = 'fred'
@@ -70,8 +71,15 @@ POSTHOOK: type: QUERY
POSTHOOK: Input: default@acid_uap
POSTHOOK: Input: default@acid_uap@ds=today
POSTHOOK: Input: default@acid_uap@ds=tomorrow
+POSTHOOK: Output: default@acid_uap
POSTHOOK: Output: default@acid_uap@ds=today
+POSTHOOK: Output: default@acid_uap@ds=today
+POSTHOOK: Output: default@acid_uap@ds=tomorrow
POSTHOOK: Output: default@acid_uap@ds=tomorrow
+POSTHOOK: Lineage: acid_uap PARTITION(ds=today).a SIMPLE [(acid_uap)acid_uap.FieldSchema(name:a, type:int, comment:null), ]
+POSTHOOK: Lineage: acid_uap PARTITION(ds=today).b EXPRESSION []
+POSTHOOK: Lineage: acid_uap PARTITION(ds=tomorrow).a SIMPLE [(acid_uap)acid_uap.FieldSchema(name:a, type:int, comment:null), ]
+POSTHOOK: Lineage: acid_uap PARTITION(ds=tomorrow).b EXPRESSION []
PREHOOK: query: select a,b,ds from acid_uap order by a,b
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_uap
diff --git a/ql/src/test/results/clientpositive/llap/update_all_types.q.out b/ql/src/test/results/clientpositive/llap/update_all_types.q.out
index 9fb8d3f5b97..3d333b5ba44 100644
--- a/ql/src/test/results/clientpositive/llap/update_all_types.q.out
+++ b/ql/src/test/results/clientpositive/llap/update_all_types.q.out
@@ -120,6 +120,7 @@ PREHOOK: query: update acid_uat set
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_uat
PREHOOK: Output: default@acid_uat
+PREHOOK: Output: default@acid_uat
POSTHOOK: query: update acid_uat set
ti = 1,
si = 2,
@@ -138,6 +139,21 @@ POSTHOOK: query: update acid_uat set
POSTHOOK: type: QUERY
POSTHOOK: Input: default@acid_uat
POSTHOOK: Output: default@acid_uat
+POSTHOOK: Output: default@acid_uat
+POSTHOOK: Lineage: acid_uat.b SIMPLE []
+POSTHOOK: Lineage: acid_uat.bi EXPRESSION []
+POSTHOOK: Lineage: acid_uat.ch EXPRESSION []
+POSTHOOK: Lineage: acid_uat.d EXPRESSION []
+POSTHOOK: Lineage: acid_uat.de EXPRESSION []
+POSTHOOK: Lineage: acid_uat.dt EXPRESSION []
+POSTHOOK: Lineage: acid_uat.f EXPRESSION []
+POSTHOOK: Lineage: acid_uat.i SIMPLE [(acid_uat)acid_uat.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: acid_uat.j SIMPLE []
+POSTHOOK: Lineage: acid_uat.s SIMPLE []
+POSTHOOK: Lineage: acid_uat.si EXPRESSION []
+POSTHOOK: Lineage: acid_uat.t EXPRESSION []
+POSTHOOK: Lineage: acid_uat.ti EXPRESSION []
+POSTHOOK: Lineage: acid_uat.vc EXPRESSION []
PREHOOK: query: select * from acid_uat order by i
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_uat
@@ -164,6 +180,7 @@ PREHOOK: query: update acid_uat set
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_uat
PREHOOK: Output: default@acid_uat
+PREHOOK: Output: default@acid_uat
POSTHOOK: query: update acid_uat set
ti = ti * 2,
si = cast(f as int),
@@ -172,6 +189,21 @@ POSTHOOK: query: update acid_uat set
POSTHOOK: type: QUERY
POSTHOOK: Input: default@acid_uat
POSTHOOK: Output: default@acid_uat
+POSTHOOK: Output: default@acid_uat
+POSTHOOK: Lineage: acid_uat.b SIMPLE [(acid_uat)acid_uat.FieldSchema(name:b, type:boolean, comment:null), ]
+POSTHOOK: Lineage: acid_uat.bi SIMPLE [(acid_uat)acid_uat.FieldSchema(name:bi, type:bigint, comment:null), ]
+POSTHOOK: Lineage: acid_uat.ch SIMPLE [(acid_uat)acid_uat.FieldSchema(name:ch, type:char(36), comment:null), ]
+POSTHOOK: Lineage: acid_uat.d EXPRESSION [(acid_uat)acid_uat.FieldSchema(name:de, type:decimal(5,2), comment:null), ]
+POSTHOOK: Lineage: acid_uat.de SIMPLE [(acid_uat)acid_uat.FieldSchema(name:de, type:decimal(5,2), comment:null), ]
+POSTHOOK: Lineage: acid_uat.dt SIMPLE [(acid_uat)acid_uat.FieldSchema(name:dt, type:date, comment:null), ]
+POSTHOOK: Lineage: acid_uat.f SIMPLE [(acid_uat)acid_uat.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: acid_uat.i SIMPLE [(acid_uat)acid_uat.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: acid_uat.j SIMPLE [(acid_uat)acid_uat.FieldSchema(name:j, type:int, comment:null), ]
+POSTHOOK: Lineage: acid_uat.s SIMPLE []
+POSTHOOK: Lineage: acid_uat.si EXPRESSION [(acid_uat)acid_uat.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: acid_uat.t SIMPLE [(acid_uat)acid_uat.FieldSchema(name:t, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: acid_uat.ti EXPRESSION [(acid_uat)acid_uat.FieldSchema(name:ti, type:tinyint, comment:null), ]
+POSTHOOK: Lineage: acid_uat.vc SIMPLE [(acid_uat)acid_uat.FieldSchema(name:vc, type:varchar(128), comment:null), ]
PREHOOK: query: select * from acid_uat order by i
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_uat
diff --git a/ql/src/test/results/clientpositive/llap/update_tmp_table.q.out b/ql/src/test/results/clientpositive/llap/update_tmp_table.q.out
index 746fb9f82ee..c15d858c710 100644
--- a/ql/src/test/results/clientpositive/llap/update_tmp_table.q.out
+++ b/ql/src/test/results/clientpositive/llap/update_tmp_table.q.out
@@ -38,10 +38,14 @@ PREHOOK: query: update acid_utt set a = 'fred' where b = '0ruyd6Y50JpdGRf6HqD'
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_utt
PREHOOK: Output: default@acid_utt
+PREHOOK: Output: default@acid_utt
POSTHOOK: query: update acid_utt set a = 'fred' where b = '0ruyd6Y50JpdGRf6HqD'
POSTHOOK: type: QUERY
POSTHOOK: Input: default@acid_utt
POSTHOOK: Output: default@acid_utt
+POSTHOOK: Output: default@acid_utt
+POSTHOOK: Lineage: acid_utt.a EXPRESSION []
+POSTHOOK: Lineage: acid_utt.b SIMPLE []
PREHOOK: query: select * from acid_utt order by a
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_utt
diff --git a/ql/src/test/results/clientpositive/llap/update_two_cols.q.out b/ql/src/test/results/clientpositive/llap/update_two_cols.q.out
index 5132c0ca0ae..5376053afb8 100644
--- a/ql/src/test/results/clientpositive/llap/update_two_cols.q.out
+++ b/ql/src/test/results/clientpositive/llap/update_two_cols.q.out
@@ -39,10 +39,15 @@ PREHOOK: query: update acid_utc set b = 'fred',c = 3.14
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_utc
PREHOOK: Output: default@acid_utc
+PREHOOK: Output: default@acid_utc
POSTHOOK: query: update acid_utc set b = 'fred',c = 3.14
POSTHOOK: type: QUERY
POSTHOOK: Input: default@acid_utc
POSTHOOK: Output: default@acid_utc
+POSTHOOK: Output: default@acid_utc
+POSTHOOK: Lineage: acid_utc.a SIMPLE [(acid_utc)acid_utc.FieldSchema(name:a, type:int, comment:null), ]
+POSTHOOK: Lineage: acid_utc.b EXPRESSION []
+POSTHOOK: Lineage: acid_utc.c EXPRESSION []
PREHOOK: query: select * from acid_utc order by a
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_utc
diff --git a/ql/src/test/results/clientpositive/llap/update_where_no_match.q.out b/ql/src/test/results/clientpositive/llap/update_where_no_match.q.out
index c88899edfe0..961010e5b7f 100644
--- a/ql/src/test/results/clientpositive/llap/update_where_no_match.q.out
+++ b/ql/src/test/results/clientpositive/llap/update_where_no_match.q.out
@@ -38,10 +38,14 @@ PREHOOK: query: update acid_wnm set b = 'fred' where b = 'nosuchvalue'
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_wnm
PREHOOK: Output: default@acid_wnm
+PREHOOK: Output: default@acid_wnm
POSTHOOK: query: update acid_wnm set b = 'fred' where b = 'nosuchvalue'
POSTHOOK: type: QUERY
POSTHOOK: Input: default@acid_wnm
POSTHOOK: Output: default@acid_wnm
+POSTHOOK: Output: default@acid_wnm
+POSTHOOK: Lineage: acid_wnm.a SIMPLE [(acid_wnm)acid_wnm.FieldSchema(name:a, type:int, comment:null), ]
+POSTHOOK: Lineage: acid_wnm.b EXPRESSION []
PREHOOK: query: select * from acid_wnm order by a
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_wnm
diff --git a/ql/src/test/results/clientpositive/llap/update_where_non_partitioned.q.out b/ql/src/test/results/clientpositive/llap/update_where_non_partitioned.q.out
index 9c79235b017..750faf20a9a 100644
--- a/ql/src/test/results/clientpositive/llap/update_where_non_partitioned.q.out
+++ b/ql/src/test/results/clientpositive/llap/update_where_non_partitioned.q.out
@@ -38,10 +38,14 @@ PREHOOK: query: update acid_uwnp set b = 'fred' where b = '0ruyd6Y50JpdGRf6HqD'
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_uwnp
PREHOOK: Output: default@acid_uwnp
+PREHOOK: Output: default@acid_uwnp
POSTHOOK: query: update acid_uwnp set b = 'fred' where b = '0ruyd6Y50JpdGRf6HqD'
POSTHOOK: type: QUERY
POSTHOOK: Input: default@acid_uwnp
POSTHOOK: Output: default@acid_uwnp
+POSTHOOK: Output: default@acid_uwnp
+POSTHOOK: Lineage: acid_uwnp.a SIMPLE [(acid_uwnp)acid_uwnp.FieldSchema(name:a, type:int, comment:null), ]
+POSTHOOK: Lineage: acid_uwnp.b EXPRESSION []
PREHOOK: query: select * from acid_uwnp order by a
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_uwnp
diff --git a/ql/src/test/results/clientpositive/llap/update_where_partitioned.q.out b/ql/src/test/results/clientpositive/llap/update_where_partitioned.q.out
index ac603b81768..ab49eeaeab5 100644
--- a/ql/src/test/results/clientpositive/llap/update_where_partitioned.q.out
+++ b/ql/src/test/results/clientpositive/llap/update_where_partitioned.q.out
@@ -63,6 +63,7 @@ PREHOOK: type: QUERY
PREHOOK: Input: default@acid_uwp
PREHOOK: Input: default@acid_uwp@ds=today
PREHOOK: Input: default@acid_uwp@ds=tomorrow
+PREHOOK: Output: default@acid_uwp
PREHOOK: Output: default@acid_uwp@ds=today
PREHOOK: Output: default@acid_uwp@ds=tomorrow
POSTHOOK: query: update acid_uwp set b = 'fred' where b = 'k17Am8uPHWk02cEf1jet'
@@ -70,8 +71,12 @@ POSTHOOK: type: QUERY
POSTHOOK: Input: default@acid_uwp
POSTHOOK: Input: default@acid_uwp@ds=today
POSTHOOK: Input: default@acid_uwp@ds=tomorrow
+POSTHOOK: Output: default@acid_uwp
+POSTHOOK: Output: default@acid_uwp@ds=today
POSTHOOK: Output: default@acid_uwp@ds=today
POSTHOOK: Output: default@acid_uwp@ds=tomorrow
+POSTHOOK: Lineage: acid_uwp PARTITION(ds=today).a SIMPLE [(acid_uwp)acid_uwp.FieldSchema(name:a, type:int, comment:null), ]
+POSTHOOK: Lineage: acid_uwp PARTITION(ds=today).b EXPRESSION []
PREHOOK: query: select * from acid_uwp order by a, ds, b
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_uwp
diff --git a/ql/src/test/results/clientpositive/llap/vector_acid4.q.out b/ql/src/test/results/clientpositive/llap/vector_acid4.q.out
index 3cdfc261d69..bad4a45a55c 100644
--- a/ql/src/test/results/clientpositive/llap/vector_acid4.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_acid4.q.out
@@ -147,22 +147,27 @@ PREHOOK: query: explain update testacid1 set id = '206' where id = '0128'
PREHOOK: type: QUERY
PREHOOK: Input: default@testacid1
PREHOOK: Output: default@testacid1
+PREHOOK: Output: default@testacid1
POSTHOOK: query: explain update testacid1 set id = '206' where id = '0128'
POSTHOOK: type: QUERY
POSTHOOK: Input: default@testacid1
POSTHOOK: Output: default@testacid1
+POSTHOOK: Output: default@testacid1
STAGE DEPENDENCIES:
- Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
- Stage-0 depends on stages: Stage-2
- Stage-3 depends on stages: Stage-0
+ Stage-2 is a root stage
+ Stage-3 depends on stages: Stage-2
+ Stage-0 depends on stages: Stage-3
+ Stage-4 depends on stages: Stage-0
+ Stage-1 depends on stages: Stage-3
+ Stage-5 depends on stages: Stage-1
STAGE PLANS:
- Stage: Stage-1
+ Stage: Stage-2
Tez
#### A masked pattern was here ####
Edges:
Reducer 2 <- Map 1 (SIMPLE_EDGE)
+ Reducer 3 <- Map 1 (SIMPLE_EDGE)
#### A masked pattern was here ####
Vertices:
Map 1
@@ -178,33 +183,63 @@ STAGE PLANS:
expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), id2 (type: string)
outputColumnNames: _col0, _col2
Statistics: Num rows: 2 Data size: 332 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: 2 Data size: 506 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: '206' (type: string), _col2 (type: string)
+ Select Operator
+ expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ outputColumnNames: _col0
+ Statistics: Num rows: 2 Data size: 152 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: 2 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: '206' (type: string), _col2 (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 2 Data size: 354 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col1 (type: string)
+ null sort order: a
+ sort order: +
+ Map-reduce partition columns: _col1 (type: string)
+ Statistics: Num rows: 2 Data size: 354 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col0 (type: string)
Execution mode: vectorized, llap
LLAP IO: may be used (ACID table)
Reducer 2
Execution mode: vectorized, llap
Reduce Operator Tree:
Select Operator
- expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), VALUE._col0 (type: string), VALUE._col1 (type: string)
- outputColumnNames: _col0, _col1, _col2
- Statistics: Num rows: 2 Data size: 506 Basic stats: COMPLETE Column stats: COMPLETE
+ expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ outputColumnNames: _col0
+ Statistics: Num rows: 2 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 2 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.testacid1
+ Write Type: DELETE
+ Reducer 3
+ Execution mode: vectorized, llap
+ Reduce Operator Tree:
+ Select Operator
+ expressions: VALUE._col0 (type: string), KEY.reducesinkkey0 (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 2 Data size: 354 Basic stats: COMPLETE Column stats: COMPLETE
File Output Operator
compressed: false
- Statistics: Num rows: 2 Data size: 506 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 2 Data size: 354 Basic stats: COMPLETE Column stats: COMPLETE
table:
input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.testacid1
- Write Type: UPDATE
+ Write Type: INSERT
- Stage: Stage-2
+ Stage: Stage-3
Dependency Collection
Stage: Stage-0
@@ -216,9 +251,24 @@ STAGE PLANS:
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.testacid1
- Write Type: UPDATE
+ Write Type: DELETE
- Stage: Stage-3
+ Stage: Stage-4
+ Stats Work
+ Basic Stats Work:
+
+ Stage: Stage-1
+ Move Operator
+ tables:
+ replace: false
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.testacid1
+ Write Type: INSERT
+
+ Stage: Stage-5
Stats Work
Basic Stats Work:
@@ -226,10 +276,14 @@ PREHOOK: query: update testacid1 set id = '206' where id = '0128'
PREHOOK: type: QUERY
PREHOOK: Input: default@testacid1
PREHOOK: Output: default@testacid1
+PREHOOK: Output: default@testacid1
POSTHOOK: query: update testacid1 set id = '206' where id = '0128'
POSTHOOK: type: QUERY
POSTHOOK: Input: default@testacid1
POSTHOOK: Output: default@testacid1
+POSTHOOK: Output: default@testacid1
+POSTHOOK: Lineage: testacid1.id SIMPLE []
+POSTHOOK: Lineage: testacid1.id2 SIMPLE [(testacid1)testacid1.FieldSchema(name:id2, type:string, comment:null), ]
PREHOOK: query: select * from testacid1 order by id limit 30
PREHOOK: type: QUERY
PREHOOK: Input: default@testacid1
diff --git a/ql/src/test/results/clientpositive/llap/vector_mapjoin_complex_values.q.out b/ql/src/test/results/clientpositive/llap/vector_mapjoin_complex_values.q.out
index 6aa762053e8..c8a992b9676 100644
--- a/ql/src/test/results/clientpositive/llap/vector_mapjoin_complex_values.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_mapjoin_complex_values.q.out
@@ -62,38 +62,43 @@ POSTHOOK: Lineage: census_clus.email SIMPLE [(census)census.FieldSchema(name:ema
POSTHOOK: Lineage: census_clus.name SIMPLE [(census)census.FieldSchema(name:name, type:string, comment:null), ]
POSTHOOK: Lineage: census_clus.ssn SIMPLE [(census)census.FieldSchema(name:ssn, type:int, comment:null), ]
census.ssn census.name census.city census.email
-Warning: Map Join MAPJOIN[22][bigTable=?] in task 'Map 1' is a cross product
+Warning: Map Join MAPJOIN[27][bigTable=?] in task 'Map 1' is a cross product
PREHOOK: query: EXPLAIN VECTORIZATION DETAIL
UPDATE census_clus SET name = 'updated name' where ssn=100 and EXISTS (select distinct ssn from census where ssn=census_clus.ssn)
PREHOOK: type: QUERY
PREHOOK: Input: default@census
PREHOOK: Input: default@census_clus
PREHOOK: Output: default@census_clus
+PREHOOK: Output: default@census_clus
POSTHOOK: query: EXPLAIN VECTORIZATION DETAIL
UPDATE census_clus SET name = 'updated name' where ssn=100 and EXISTS (select distinct ssn from census where ssn=census_clus.ssn)
POSTHOOK: type: QUERY
POSTHOOK: Input: default@census
POSTHOOK: Input: default@census_clus
POSTHOOK: Output: default@census_clus
+POSTHOOK: Output: default@census_clus
Explain
PLAN VECTORIZATION:
enabled: true
enabledConditionsMet: [hive.vectorized.execution.enabled IS true]
STAGE DEPENDENCIES:
- Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
- Stage-0 depends on stages: Stage-2
- Stage-3 depends on stages: Stage-0
+ Stage-2 is a root stage
+ Stage-3 depends on stages: Stage-2
+ Stage-0 depends on stages: Stage-3
+ Stage-4 depends on stages: Stage-0
+ Stage-1 depends on stages: Stage-3
+ Stage-5 depends on stages: Stage-1
STAGE PLANS:
- Stage: Stage-1
+ Stage: Stage-2
Tez
#### A masked pattern was here ####
Edges:
- Map 1 <- Reducer 4 (BROADCAST_EDGE)
+ Map 1 <- Reducer 5 (BROADCAST_EDGE)
Reducer 2 <- Map 1 (SIMPLE_EDGE)
- Reducer 4 <- Map 3 (SIMPLE_EDGE)
+ Reducer 3 <- Map 1 (SIMPLE_EDGE)
+ Reducer 5 <- Map 4 (SIMPLE_EDGE)
#### A masked pattern was here ####
Vertices:
Map 1
@@ -134,7 +139,7 @@ STAGE PLANS:
nativeConditionsNotMet: Supports Value Types [STRUCT] IS false
outputColumnNames: _col1, _col2, _col3
input vertices:
- 1 Reducer 4
+ 1 Reducer 5
Statistics: Num rows: 1 Data size: 257 Basic stats: COMPLETE Column stats: COMPLETE
Select Operator
expressions: _col3 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), _col1 (type: string), _col2 (type: string)
@@ -144,21 +149,49 @@ STAGE PLANS:
native: true
projectedOutputColumnNums: [2, 0, 1]
Statistics: Num rows: 1 Data size: 257 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)
- Reduce Sink Vectorization:
- className: VectorReduceSinkObjectHashOperator
- keyColumns: 2:struct<writeid:bigint,bucketid:int,rowid:bigint>
+ Select Operator
+ expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ outputColumnNames: _col0
+ Select Vectorization:
+ className: VectorSelectOperator
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
- partitionColumns: 5:int
- valueColumns: 3:int, 4:string, 0:string, 1:string
- valueExpressions: ConstantVectorExpression(val 100) -> 3:int, ConstantVectorExpression(val updated name) -> 4:string
- Statistics: Num rows: 1 Data size: 357 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: 100 (type: int), 'updated name' (type: string), _col3 (type: string), _col4 (type: string)
+ projectedOutputColumnNums: [2]
+ 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)
+ Reduce Sink Vectorization:
+ className: VectorReduceSinkObjectHashOperator
+ keyColumns: 2:struct<writeid:bigint,bucketid:int,rowid:bigint>
+ 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
+ partitionColumns: 5:int
+ Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: 100 (type: int), 'updated name' (type: string), _col3 (type: string), _col4 (type: string)
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Select Vectorization:
+ className: VectorSelectOperator
+ native: true
+ projectedOutputColumnNums: [3, 4, 0, 1]
+ selectExpressions: ConstantVectorExpression(val 100) -> 3:int, ConstantVectorExpression(val updated name) -> 4:string
+ Statistics: Num rows: 1 Data size: 281 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)
+ Reduce Sink Vectorization:
+ className: VectorReduceSinkObjectHashOperator
+ keyColumns: 3:int
+ 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
+ partitionColumns: 3:int
+ valueColumns: 4:string, 0:string, 1:string
+ Statistics: Num rows: 1 Data size: 281 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string)
Execution mode: vectorized, llap
LLAP IO: may be used (ACID table)
Map Vectorization:
@@ -177,7 +210,7 @@ STAGE PLANS:
neededVirtualColumns: [ROWID]
partitionColumnCount: 0
scratchColumnTypeNames: []
- Map 3
+ Map 4
Map Operator Tree:
TableScan
alias: census
@@ -251,32 +284,68 @@ STAGE PLANS:
usesVectorUDFAdaptor: false
vectorized: true
rowBatchContext:
- dataColumnCount: 5
- dataColumns: KEY.reducesinkkey0:struct<writeid:bigint,bucketid:int,rowid:bigint>, VALUE._col0:int, VALUE._col1:string, VALUE._col2:string, VALUE._col3:string
+ dataColumnCount: 1
+ dataColumns: KEY.reducesinkkey0:struct<writeid:bigint,bucketid:int,rowid:bigint>
+ partitionColumnCount: 0
+ scratchColumnTypeNames: []
+ Reduce Operator Tree:
+ Select Operator
+ expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ outputColumnNames: _col0
+ Select Vectorization:
+ className: VectorSelectOperator
+ native: true
+ projectedOutputColumnNums: [0]
+ Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: COMPLETE
+ File Output Operator
+ compressed: false
+ File Sink Vectorization:
+ className: VectorFileSinkOperator
+ native: false
+ Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: COMPLETE
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.census_clus
+ Write Type: DELETE
+ Reducer 3
+ Execution mode: vectorized, llap
+ Reduce Vectorization:
+ enabled: true
+ enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez] IS true
+ reduceColumnNullOrder: a
+ reduceColumnSortOrder: +
+ allNative: false
+ usesVectorUDFAdaptor: false
+ vectorized: true
+ rowBatchContext:
+ dataColumnCount: 4
+ dataColumns: KEY.reducesinkkey0:int, VALUE._col0:string, VALUE._col1:string, VALUE._col2:string
partitionColumnCount: 0
scratchColumnTypeNames: []
Reduce Operator Tree:
Select Operator
- expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: string)
- outputColumnNames: _col0, _col1, _col2, _col3, _col4
+ expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string)
+ outputColumnNames: _col0, _col1, _col2, _col3
Select Vectorization:
className: VectorSelectOperator
native: true
- projectedOutputColumnNums: [0, 1, 2, 3, 4]
- Statistics: Num rows: 1 Data size: 357 Basic stats: COMPLETE Column stats: COMPLETE
+ projectedOutputColumnNums: [0, 1, 2, 3]
+ Statistics: Num rows: 1 Data size: 281 Basic stats: COMPLETE Column stats: COMPLETE
File Output Operator
compressed: false
File Sink Vectorization:
className: VectorFileSinkOperator
native: false
- Statistics: Num rows: 1 Data size: 357 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 1 Data size: 281 Basic stats: COMPLETE Column stats: COMPLETE
table:
input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.census_clus
- Write Type: UPDATE
- Reducer 4
+ Write Type: INSERT
+ Reducer 5
Execution mode: vectorized, llap
Reduce Vectorization:
enabled: true
@@ -319,7 +388,7 @@ STAGE PLANS:
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
Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
- Stage: Stage-2
+ Stage: Stage-3
Dependency Collection
Stage: Stage-0
@@ -331,21 +400,42 @@ STAGE PLANS:
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.census_clus
- Write Type: UPDATE
+ Write Type: DELETE
- Stage: Stage-3
+ Stage: Stage-4
+ Stats Work
+ Basic Stats Work:
+
+ Stage: Stage-1
+ Move Operator
+ tables:
+ replace: false
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.census_clus
+ Write Type: INSERT
+
+ Stage: Stage-5
Stats Work
Basic Stats Work:
-Warning: Map Join MAPJOIN[22][bigTable=?] in task 'Map 1' is a cross product
+Warning: Map Join MAPJOIN[27][bigTable=?] in task 'Map 1' is a cross product
PREHOOK: query: UPDATE census_clus SET name = 'updated name' where ssn=100 and EXISTS (select distinct ssn from census where ssn=census_clus.ssn)
PREHOOK: type: QUERY
PREHOOK: Input: default@census
PREHOOK: Input: default@census_clus
PREHOOK: Output: default@census_clus
+PREHOOK: Output: default@census_clus
POSTHOOK: query: UPDATE census_clus SET name = 'updated name' where ssn=100 and EXISTS (select distinct ssn from census where ssn=census_clus.ssn)
POSTHOOK: type: QUERY
POSTHOOK: Input: default@census
POSTHOOK: Input: default@census_clus
POSTHOOK: Output: default@census_clus
-row__id ssn _c2 city email
+POSTHOOK: Output: default@census_clus
+POSTHOOK: Lineage: census_clus.city SIMPLE [(census_clus)census_clus.FieldSchema(name:city, type:string, comment:null), ]
+POSTHOOK: Lineage: census_clus.email SIMPLE [(census_clus)census_clus.FieldSchema(name:email, type:string, comment:null), ]
+POSTHOOK: Lineage: census_clus.name SIMPLE []
+POSTHOOK: Lineage: census_clus.ssn SIMPLE []
+s.ssn s.name s.city s.email
diff --git a/ql/src/test/results/clientpositive/tez/acid_vectorization_original_tez.q.out b/ql/src/test/results/clientpositive/tez/acid_vectorization_original_tez.q.out
index b894b90bc66..a4ab46ea764 100644
--- a/ql/src/test/results/clientpositive/tez/acid_vectorization_original_tez.q.out
+++ b/ql/src/test/results/clientpositive/tez/acid_vectorization_original_tez.q.out
@@ -581,22 +581,27 @@ PREHOOK: query: explain update over10k_orc_bucketed_n0 set i = 0 where b = 42949
PREHOOK: type: QUERY
PREHOOK: Input: default@over10k_orc_bucketed_n0
PREHOOK: Output: default@over10k_orc_bucketed_n0
+PREHOOK: Output: default@over10k_orc_bucketed_n0
POSTHOOK: query: explain update over10k_orc_bucketed_n0 set i = 0 where b = 4294967363 and t < 100
POSTHOOK: type: QUERY
POSTHOOK: Input: default@over10k_orc_bucketed_n0
POSTHOOK: Output: default@over10k_orc_bucketed_n0
+POSTHOOK: Output: default@over10k_orc_bucketed_n0
STAGE DEPENDENCIES:
- Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
- Stage-0 depends on stages: Stage-2
- Stage-3 depends on stages: Stage-0
+ Stage-2 is a root stage
+ Stage-3 depends on stages: Stage-2
+ Stage-0 depends on stages: Stage-3
+ Stage-4 depends on stages: Stage-0
+ Stage-1 depends on stages: Stage-3
+ Stage-5 depends on stages: Stage-1
STAGE PLANS:
- Stage: Stage-1
+ Stage: Stage-2
Tez
#### A masked pattern was here ####
Edges:
Reducer 2 <- Map 1 (SIMPLE_EDGE)
+ Reducer 3 <- Map 1 (SIMPLE_EDGE)
#### A masked pattern was here ####
Vertices:
Map 1
@@ -612,32 +617,62 @@ STAGE PLANS:
expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), t (type: tinyint), si (type: smallint), f (type: float), d (type: double), bo (type: boolean), s (type: string), ts (type: timestamp), dec (type: decimal(4,2)), bin (type: binary)
outputColumnNames: _col0, _col1, _col2, _col5, _col6, _col7, _col8, _col9, _col10, _col11
Statistics: Num rows: 7 Data size: 2835 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: 7 Data size: 2919 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: _col1 (type: tinyint), _col2 (type: smallint), 0 (type: int), 4294967363L (type: bigint), _col5 (type: float), _col6 (type: double), _col7 (type: boolean), _col8 (type: string), _col9 (type: timestamp), _col10 (type: decimal(4,2)), _col11 (type: binary)
+ Select Operator
+ expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ outputColumnNames: _col0
+ Statistics: Num rows: 7 Data size: 532 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: 7 Data size: 532 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: _col1 (type: tinyint), _col2 (type: smallint), 0 (type: int), 4294967363L (type: bigint), _col5 (type: float), _col6 (type: double), _col7 (type: boolean), _col8 (type: string), _col9 (type: timestamp), _col10 (type: decimal(4,2)), _col11 (type: binary)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
+ Statistics: Num rows: 7 Data size: 2387 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col1 (type: smallint)
+ null sort order: a
+ sort order: +
+ Map-reduce partition columns: _col1 (type: smallint)
+ Statistics: Num rows: 7 Data size: 2387 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col0 (type: tinyint), _col2 (type: int), _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: boolean), _col7 (type: string), _col8 (type: timestamp), _col9 (type: decimal(4,2)), _col10 (type: binary)
Execution mode: vectorized
Reducer 2
Execution mode: vectorized
Reduce Operator Tree:
Select Operator
- expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), VALUE._col0 (type: tinyint), VALUE._col1 (type: smallint), VALUE._col2 (type: int), VALUE._col3 (type: bigint), VALUE._col4 (type: float), VALUE._col5 (type: double), VALUE._col6 (type: boolean), VALUE._col7 (type: string), VALUE._col8 (type: timestamp), VALUE._col9 (type: decimal(4,2)), VALUE._col10 (type: binary)
- outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
- Statistics: Num rows: 7 Data size: 2919 Basic stats: COMPLETE Column stats: COMPLETE
+ expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
+ outputColumnNames: _col0
+ Statistics: Num rows: 7 Data size: 532 Basic stats: COMPLETE Column stats: COMPLETE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 7 Data size: 532 Basic stats: COMPLETE Column stats: COMPLETE
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.over10k_orc_bucketed_n0
+ Write Type: DELETE
+ Reducer 3
+ Execution mode: vectorized
+ Reduce Operator Tree:
+ Select Operator
+ expressions: VALUE._col0 (type: tinyint), KEY.reducesinkkey0 (type: smallint), VALUE._col1 (type: int), VALUE._col2 (type: bigint), VALUE._col3 (type: float), VALUE._col4 (type: double), VALUE._col5 (type: boolean), VALUE._col6 (type: string), VALUE._col7 (type: timestamp), VALUE._col8 (type: decimal(4,2)), VALUE._col9 (type: binary)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
+ Statistics: Num rows: 7 Data size: 2387 Basic stats: COMPLETE Column stats: COMPLETE
File Output Operator
compressed: false
- Statistics: Num rows: 7 Data size: 2919 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 7 Data size: 2387 Basic stats: COMPLETE Column stats: COMPLETE
table:
input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.over10k_orc_bucketed_n0
- Write Type: UPDATE
+ Write Type: INSERT
- Stage: Stage-2
+ Stage: Stage-3
Dependency Collection
Stage: Stage-0
@@ -649,9 +684,24 @@ STAGE PLANS:
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.over10k_orc_bucketed_n0
- Write Type: UPDATE
+ Write Type: DELETE
- Stage: Stage-3
+ Stage: Stage-4
+ Stats Work
+ Basic Stats Work:
+
+ Stage: Stage-1
+ Move Operator
+ tables:
+ replace: false
+ table:
+ input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+ serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+ name: default.over10k_orc_bucketed_n0
+ Write Type: INSERT
+
+ Stage: Stage-5
Stats Work
Basic Stats Work:
@@ -659,10 +709,23 @@ PREHOOK: query: update over10k_orc_bucketed_n0 set i = 0 where b = 4294967363 an
PREHOOK: type: QUERY
PREHOOK: Input: default@over10k_orc_bucketed_n0
PREHOOK: Output: default@over10k_orc_bucketed_n0
+PREHOOK: Output: default@over10k_orc_bucketed_n0
POSTHOOK: query: update over10k_orc_bucketed_n0 set i = 0 where b = 4294967363 and t < 100
POSTHOOK: type: QUERY
POSTHOOK: Input: default@over10k_orc_bucketed_n0
POSTHOOK: Output: default@over10k_orc_bucketed_n0
+POSTHOOK: Output: default@over10k_orc_bucketed_n0
+POSTHOOK: Lineage: over10k_orc_bucketed_n0.b SIMPLE []
+POSTHOOK: Lineage: over10k_orc_bucketed_n0.bin SIMPLE [(over10k_orc_bucketed_n0)over10k_orc_bucketed_n0.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over10k_orc_bucketed_n0.bo SIMPLE [(over10k_orc_bucketed_n0)over10k_orc_bucketed_n0.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over10k_orc_bucketed_n0.d SIMPLE [(over10k_orc_bucketed_n0)over10k_orc_bucketed_n0.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over10k_orc_bucketed_n0.dec SIMPLE [(over10k_orc_bucketed_n0)over10k_orc_bucketed_n0.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over10k_orc_bucketed_n0.f SIMPLE [(over10k_orc_bucketed_n0)over10k_orc_bucketed_n0.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over10k_orc_bucketed_n0.i SIMPLE []
+POSTHOOK: Lineage: over10k_orc_bucketed_n0.s SIMPLE [(over10k_orc_bucketed_n0)over10k_orc_bucketed_n0.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over10k_orc_bucketed_n0.si SIMPLE [(over10k_orc_bucketed_n0)over10k_orc_bucketed_n0.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over10k_orc_bucketed_n0.t SIMPLE [(over10k_orc_bucketed_n0)over10k_orc_bucketed_n0.FieldSchema(name:t, type:tinyint, comment:null), ]
+POSTHOOK: Lineage: over10k_orc_bucketed_n0.ts SIMPLE [(over10k_orc_bucketed_n0)over10k_orc_bucketed_n0.FieldSchema(name:ts, type:timestamp, comment:null), ]
PREHOOK: query: select ROW__ID, t, si, i from over10k_orc_bucketed_n0 where b = 4294967363 and t < 100 order by ROW__ID
PREHOOK: type: QUERY
PREHOOK: Input: default@over10k_orc_bucketed_n0
@@ -671,12 +734,12 @@ POSTHOOK: query: select ROW__ID, t, si, i from over10k_orc_bucketed_n0 where b =
POSTHOOK: type: QUERY
POSTHOOK: Input: default@over10k_orc_bucketed_n0
POSTHOOK: Output: hdfs://### HDFS PATH ###
-{"writeid":### Masked writeid ###,"bucketid":536870912,"rowid":0} 5 501 0
-{"writeid":### Masked writeid ###,"bucketid":536870912,"rowid":1} 5 501 0
-{"writeid":### Masked writeid ###,"bucketid":536936448,"rowid":0} -3 344 0
-{"writeid":### Masked writeid ###,"bucketid":536936448,"rowid":1} -3 344 0
-{"writeid":### Masked writeid ###,"bucketid":537067520,"rowid":0} 35 463 0
-{"writeid":### Masked writeid ###,"bucketid":537067520,"rowid":1} 35 463 0
+{"writeid":### Masked writeid ###,"bucketid":536870913,"rowid":0} 5 501 0
+{"writeid":### Masked writeid ###,"bucketid":536870913,"rowid":1} 5 501 0
+{"writeid":### Masked writeid ###,"bucketid":536936449,"rowid":0} -3 344 0
+{"writeid":### Masked writeid ###,"bucketid":536936449,"rowid":1} -3 344 0
+{"writeid":### Masked writeid ###,"bucketid":537067521,"rowid":0} 35 463 0
+{"writeid":### Masked writeid ###,"bucketid":537067521,"rowid":1} 35 463 0
PREHOOK: query: explain select ROW__ID, count(*) from over10k_orc_bucketed_n0 group by ROW__ID having count(*) > 1
PREHOOK: type: QUERY
PREHOOK: Input: default@over10k_orc_bucketed_n0
@@ -782,12 +845,12 @@ POSTHOOK: query: select ROW__ID, t, si, i from over10k_orc_bucketed_n0 where b =
POSTHOOK: type: QUERY
POSTHOOK: Input: default@over10k_orc_bucketed_n0
POSTHOOK: Output: hdfs://### HDFS PATH ###
-{"writeid":### Masked writeid ###,"bucketid":536870912,"rowid":0} 5 501 0
-{"writeid":### Masked writeid ###,"bucketid":536870912,"rowid":1} 5 501 0
-{"writeid":### Masked writeid ###,"bucketid":536936448,"rowid":0} -3 344 0
-{"writeid":### Masked writeid ###,"bucketid":536936448,"rowid":1} -3 344 0
-{"writeid":### Masked writeid ###,"bucketid":537067520,"rowid":0} 35 463 0
-{"writeid":### Masked writeid ###,"bucketid":537067520,"rowid":1} 35 463 0
+{"writeid":### Masked writeid ###,"bucketid":536870913,"rowid":0} 5 501 0
+{"writeid":### Masked writeid ###,"bucketid":536870913,"rowid":1} 5 501 0
+{"writeid":### Masked writeid ###,"bucketid":536936449,"rowid":0} -3 344 0
+{"writeid":### Masked writeid ###,"bucketid":536936449,"rowid":1} -3 344 0
+{"writeid":### Masked writeid ###,"bucketid":537067521,"rowid":0} 35 463 0
+{"writeid":### Masked writeid ###,"bucketid":537067521,"rowid":1} 35 463 0
PREHOOK: query: select ROW__ID, count(*) from over10k_orc_bucketed_n0 group by ROW__ID having count(*) > 1
PREHOOK: type: QUERY
PREHOOK: Input: default@over10k_orc_bucketed_n0
diff --git a/ql/src/test/results/clientpositive/tez/explainanalyze_5.q.out b/ql/src/test/results/clientpositive/tez/explainanalyze_5.q.out
index 5276cc49f50..aaf7f733724 100644
--- a/ql/src/test/results/clientpositive/tez/explainanalyze_5.q.out
+++ b/ql/src/test/results/clientpositive/tez/explainanalyze_5.q.out
@@ -293,43 +293,67 @@ PREHOOK: query: update acid_uami_n2 set de = 3.14 where de = 109.23 or de = 119.
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_uami_n2
PREHOOK: Output: default@acid_uami_n2
+PREHOOK: Output: default@acid_uami_n2
POSTHOOK: query: update acid_uami_n2 set de = 3.14 where de = 109.23 or de = 119.23
POSTHOOK: type: QUERY
POSTHOOK: Input: default@acid_uami_n2
POSTHOOK: Output: default@acid_uami_n2
+POSTHOOK: Output: default@acid_uami_n2
PREHOOK: query: explain analyze update acid_uami_n2 set de = 3.14 where de = 109.23 or de = 119.23
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_uami_n2
PREHOOK: Output: default@acid_uami_n2
+PREHOOK: Output: default@acid_uami_n2
POSTHOOK: query: explain analyze update acid_uami_n2 set de = 3.14 where de = 109.23 or de = 119.23
POSTHOOK: type: QUERY
POSTHOOK: Input: default@acid_uami_n2
POSTHOOK: Output: default@acid_uami_n2
+POSTHOOK: Output: default@acid_uami_n2
Vertex dependency in root stage
Reducer 2 <- Map 1 (SIMPLE_EDGE)
+Reducer 3 <- Map 1 (SIMPLE_EDGE)
-Stage-3
+Stage-4
Stats Work{}
Stage-0
Move Operator
table:{"name:":"default.acid_uami_n2"}
- Stage-2
+ Stage-3
Dependency Collection{}
- Stage-1
+ Stage-2
Reducer 2
File Output Operator [FS_8]
table:{"name:":"default.acid_uami_n2"}
- Select Operator [SEL_7] (rows=2/2 width=302)
- Output:["_col0","_col1","_col2","_col3"]
+ Select Operator [SEL_7] (rows=2/2 width=76)
+ Output:["_col0"]
<-Map 1 [SIMPLE_EDGE]
SHUFFLE [RS_6]
PartitionCols:UDFToInteger(_col0)
- Select Operator [SEL_2] (rows=2/2 width=190)
- Output:["_col0","_col1","_col3"]
- Filter Operator [FIL_9] (rows=2/2 width=226)
- predicate:(de) IN (109.23, 119.23)
- TableScan [TS_0] (rows=4/2 width=226)
- default@acid_uami_n2,acid_uami_n2, ACID table,Tbl:COMPLETE,Col:COMPLETE,Output:["i","de","vc"]
+ Select Operator [SEL_3] (rows=2/2 width=76)
+ Output:["_col0"]
+ Select Operator [SEL_2] (rows=2/2 width=190)
+ Output:["_col0","_col1","_col3"]
+ Filter Operator [FIL_14] (rows=2/2 width=226)
+ predicate:(de) IN (109.23, 119.23)
+ TableScan [TS_0] (rows=4/2 width=226)
+ default@acid_uami_n2,acid_uami_n2, ACID table,Tbl:COMPLETE,Col:COMPLETE,Output:["i","de","vc"]
+ Reducer 3
+ File Output Operator [FS_13]
+ table:{"name:":"default.acid_uami_n2"}
+ Select Operator [SEL_12] (rows=2/2 width=226)
+ Output:["_col0","_col1","_col2"]
+ <-Map 1 [SIMPLE_EDGE]
+ SHUFFLE [RS_11]
+ PartitionCols:_col0
+ Select Operator [SEL_9] (rows=2/2 width=226)
+ Output:["_col0","_col1","_col2"]
+ Please refer to the previous Select Operator [SEL_2]
+Stage-5
+ Stats Work{}
+ Stage-1
+ Move Operator
+ table:{"name:":"default.acid_uami_n2"}
+ Please refer to the previous Stage-3
PREHOOK: query: select * from acid_uami_n2 order by de
PREHOOK: type: QUERY
@@ -347,10 +371,15 @@ PREHOOK: query: update acid_uami_n2 set de = 3.14 where de = 109.23 or de = 119.
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_uami_n2
PREHOOK: Output: default@acid_uami_n2
+PREHOOK: Output: default@acid_uami_n2
POSTHOOK: query: update acid_uami_n2 set de = 3.14 where de = 109.23 or de = 119.23
POSTHOOK: type: QUERY
POSTHOOK: Input: default@acid_uami_n2
POSTHOOK: Output: default@acid_uami_n2
+POSTHOOK: Output: default@acid_uami_n2
+POSTHOOK: Lineage: acid_uami_n2.de EXPRESSION []
+POSTHOOK: Lineage: acid_uami_n2.i SIMPLE [(acid_uami_n2)acid_uami_n2.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: acid_uami_n2.vc SIMPLE [(acid_uami_n2)acid_uami_n2.FieldSchema(name:vc, type:varchar(128), comment:null), ]
PREHOOK: query: select * from acid_uami_n2 order by de
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_uami_n2
diff --git a/ql/src/test/results/clientpositive/tez/update_orig_table.q.out b/ql/src/test/results/clientpositive/tez/update_orig_table.q.out
index b4ea6ee69b1..cd1b74f9053 100644
--- a/ql/src/test/results/clientpositive/tez/update_orig_table.q.out
+++ b/ql/src/test/results/clientpositive/tez/update_orig_table.q.out
@@ -52,10 +52,24 @@ PREHOOK: query: update acid_uot set cstring1 = 'fred' where cint < -1070551679
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_uot
PREHOOK: Output: default@acid_uot
+PREHOOK: Output: default@acid_uot
POSTHOOK: query: update acid_uot set cstring1 = 'fred' where cint < -1070551679
POSTHOOK: type: QUERY
POSTHOOK: Input: default@acid_uot
POSTHOOK: Output: default@acid_uot
+POSTHOOK: Output: default@acid_uot
+POSTHOOK: Lineage: acid_uot.cbigint SIMPLE [(acid_uot)acid_uot.FieldSchema(name:cbigint, type:bigint, comment:null), ]
+POSTHOOK: Lineage: acid_uot.cboolean1 SIMPLE [(acid_uot)acid_uot.FieldSchema(name:cboolean1, type:boolean, comment:null), ]
+POSTHOOK: Lineage: acid_uot.cboolean2 SIMPLE [(acid_uot)acid_uot.FieldSchema(name:cboolean2, type:boolean, comment:null), ]
+POSTHOOK: Lineage: acid_uot.cdouble SIMPLE [(acid_uot)acid_uot.FieldSchema(name:cdouble, type:double, comment:null), ]
+POSTHOOK: Lineage: acid_uot.cfloat SIMPLE [(acid_uot)acid_uot.FieldSchema(name:cfloat, type:float, comment:null), ]
+POSTHOOK: Lineage: acid_uot.cint SIMPLE [(acid_uot)acid_uot.FieldSchema(name:cint, type:int, comment:null), ]
+POSTHOOK: Lineage: acid_uot.csmallint SIMPLE [(acid_uot)acid_uot.FieldSchema(name:csmallint, type:smallint, comment:null), ]
+POSTHOOK: Lineage: acid_uot.cstring1 SIMPLE []
+POSTHOOK: Lineage: acid_uot.cstring2 SIMPLE [(acid_uot)acid_uot.FieldSchema(name:cstring2, type:string, comment:null), ]
+POSTHOOK: Lineage: acid_uot.ctimestamp1 SIMPLE [(acid_uot)acid_uot.FieldSchema(name:ctimestamp1, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: acid_uot.ctimestamp2 SIMPLE [(acid_uot)acid_uot.FieldSchema(name:ctimestamp2, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: acid_uot.ctinyint SIMPLE [(acid_uot)acid_uot.FieldSchema(name:ctinyint, type:tinyint, comment:null), ]
PREHOOK: query: select * from acid_uot where cstring1 = 'fred'
PREHOOK: type: QUERY
PREHOOK: Input: default@acid_uot
diff --git a/streaming/src/test/org/apache/hive/streaming/TestStreaming.java b/streaming/src/test/org/apache/hive/streaming/TestStreaming.java
index d8b5e0e7ed3..3655bf236af 100644
--- a/streaming/src/test/org/apache/hive/streaming/TestStreaming.java
+++ b/streaming/src/test/org/apache/hive/streaming/TestStreaming.java
@@ -422,7 +422,7 @@ public class TestStreaming {
Assert.assertTrue(rs.get(1), rs.get(1).endsWith("streamingnobuckets/base_0000005_v0000024/bucket_00000"));
Assert.assertTrue(rs.get(2), rs.get(2).startsWith("{\"writeid\":3,\"bucketid\":536870912,\"rowid\":0}\ta5\tb6"));
Assert.assertTrue(rs.get(2), rs.get(2).endsWith("streamingnobuckets/base_0000005_v0000024/bucket_00000"));
- Assert.assertTrue(rs.get(3), rs.get(3).startsWith("{\"writeid\":4,\"bucketid\":536870912,\"rowid\":0}\t0\t0"));
+ Assert.assertTrue(rs.get(3), rs.get(3).startsWith("{\"writeid\":4,\"bucketid\":536870913,\"rowid\":0}\t0\t0"));
Assert.assertTrue(rs.get(3), rs.get(3).endsWith("streamingnobuckets/base_0000005_v0000024/bucket_00000"));
}
@@ -776,8 +776,8 @@ public class TestStreaming {
Assert.assertTrue(rs.get(3), rs.get(3).endsWith("streamingnobuckets/base_0000009_v0000028/bucket_00000"));
Assert.assertTrue(rs.get(4), rs.get(4).startsWith("{\"writeid\":5,\"bucketid\":536870912,\"rowid\":0}\ta13\tb14"));
Assert.assertTrue(rs.get(4), rs.get(4).endsWith("streamingnobuckets/base_0000009_v0000028/bucket_00000"));
- Assert.assertTrue(rs.get(5), rs.get(5).startsWith("{\"writeid\":6,\"bucketid\":536870912,\"rowid\":0}\t0\t0"));
- Assert.assertTrue(rs.get(5), rs.get(5).endsWith("streamingnobuckets/base_0000009_v0000028/bucket_00000"));
+ Assert.assertTrue(rs.get(5), rs.get(5).startsWith("{\"writeid\":6,\"bucketid\":536936449,\"rowid\":0}\t0\t0"));
+ Assert.assertTrue(rs.get(5), rs.get(5).endsWith("streamingnobuckets/base_0000009_v0000028/bucket_00001"));
}
/**