You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by se...@apache.org on 2016/01/27 23:47:27 UTC

[14/14] hive git commit: HIVE-12523 : display Hive query name in explain plan (Sergey Shelukhin, reviewed by Gunther Hagleitner)

HIVE-12523 : display Hive query name in explain plan (Sergey Shelukhin, reviewed by Gunther Hagleitner)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/cbfd2570
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/cbfd2570
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/cbfd2570

Branch: refs/heads/branch-2.0
Commit: cbfd25704c1f6a743575f4432f8100c2bd949174
Parents: d9cb3e4
Author: Sergey Shelukhin <se...@apache.org>
Authored: Wed Jan 27 14:29:42 2016 -0800
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Wed Jan 27 14:42:27 2016 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/ql/QTestUtil.java    |   1 +
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |   2 +-
 .../hadoop/hive/ql/exec/tez/DagUtils.java       |  13 +-
 .../hive/ql/optimizer/GenMapRedUtils.java       |   4 +-
 .../hadoop/hive/ql/parse/GenTezProcContext.java |   2 +-
 .../org/apache/hadoop/hive/ql/plan/TezWork.java |  17 ++-
 .../hadoop/hive/ql/exec/tez/TestTezTask.java    |   4 +-
 .../apache/hadoop/hive/ql/plan/TestTezWork.java |   2 +-
 .../llap/bucket_map_join_tez1.q.out             |  15 +++
 .../llap/bucket_map_join_tez2.q.out             |   7 ++
 .../llap/dynamic_partition_pruning.q.out        |  42 +++++++
 .../llap/dynamic_partition_pruning_2.q.out      |   7 ++
 .../llap/hybridgrace_hashjoin_1.q.out           |  12 ++
 .../llap/hybridgrace_hashjoin_2.q.out           |   8 ++
 .../clientpositive/llap/llapdecider.q.out       |  16 ++-
 .../clientpositive/llap/lvj_mapjoin.q.out       |   1 +
 .../clientpositive/llap/mapjoin_decimal.q.out   |   1 +
 .../test/results/clientpositive/llap/mrr.q.out  |   6 +-
 .../llap/tez_bmj_schema_evolution.q.out         |   1 +
 .../results/clientpositive/llap/tez_dml.q.out   |   2 +-
 .../llap/tez_dynpart_hashjoin_1.q.out           |   7 +-
 .../llap/tez_dynpart_hashjoin_2.q.out           |   4 +-
 .../results/clientpositive/llap/tez_join.q.out  |   1 +
 .../clientpositive/llap/tez_join_hash.q.out     |   2 +
 .../llap/tez_join_result_complex.q.out          |   2 +
 .../clientpositive/llap/tez_join_tests.q.out    |   2 +-
 .../clientpositive/llap/tez_joins_explain.q.out |   2 +-
 .../clientpositive/llap/tez_self_join.q.out     |   1 +
 .../results/clientpositive/llap/tez_smb_1.q.out |   4 +
 .../clientpositive/llap/tez_smb_main.q.out      |  12 +-
 .../results/clientpositive/llap/tez_union.q.out |   8 +-
 .../clientpositive/llap/tez_union2.q.out        |   2 +-
 .../llap/tez_union_dynamic_partition.q.out      |   1 +
 .../llap/tez_union_group_by.q.out               |   1 +
 .../llap/tez_union_multiinsert.q.out            |   5 +
 .../llap/tez_vector_dynpart_hashjoin_1.q.out    |   7 +-
 .../llap/tez_vector_dynpart_hashjoin_2.q.out    |   4 +-
 .../llap/vector_join_part_col_char.q.out        |   1 +
 .../vectorized_dynamic_partition_pruning.q.out  |  39 ++++++
 .../results/clientpositive/tez/auto_join0.q.out |   2 +-
 .../results/clientpositive/tez/auto_join1.q.out |   1 +
 .../clientpositive/tez/auto_join21.q.out        |   2 +-
 .../clientpositive/tez/auto_join29.q.out        |  10 +-
 .../clientpositive/tez/auto_join30.q.out        |   9 +-
 .../tez/auto_sortmerge_join_1.q.out             |   3 +
 .../tez/auto_sortmerge_join_10.q.out            |   2 +
 .../tez/auto_sortmerge_join_11.q.out            |   4 +
 .../tez/auto_sortmerge_join_12.q.out            |   1 +
 .../tez/auto_sortmerge_join_13.q.out            |   3 +
 .../tez/auto_sortmerge_join_14.q.out            |   2 +
 .../tez/auto_sortmerge_join_15.q.out            |   2 +
 .../tez/auto_sortmerge_join_2.q.out             |   2 +
 .../tez/auto_sortmerge_join_3.q.out             |   3 +
 .../tez/auto_sortmerge_join_4.q.out             |   3 +
 .../tez/auto_sortmerge_join_5.q.out             |   3 +
 .../tez/auto_sortmerge_join_6.q.out             |  10 ++
 .../tez/auto_sortmerge_join_7.q.out             |   3 +
 .../tez/auto_sortmerge_join_8.q.out             |   3 +
 .../tez/auto_sortmerge_join_9.q.out             |  25 ++++
 .../results/clientpositive/tez/bucket2.q.out    |   1 +
 .../results/clientpositive/tez/bucket3.q.out    |   1 +
 .../results/clientpositive/tez/bucket4.q.out    |   1 +
 .../tez/bucket_map_join_tez1.q.out              |  15 +++
 .../tez/bucket_map_join_tez2.q.out              |   7 ++
 .../tez/correlationoptimizer1.q.out             |  22 +++-
 .../test/results/clientpositive/tez/count.q.out |   4 +
 .../results/clientpositive/tez/cross_join.q.out |   7 +-
 .../tez/cross_product_check_1.q.out             |   5 +
 .../tez/cross_product_check_2.q.out             |   5 +
 .../test/results/clientpositive/tez/ctas.q.out  |   5 +
 .../tez/disable_merge_for_bucketing.q.out       |   2 +
 .../tez/dynamic_partition_pruning.q.out         |  42 +++++++
 .../tez/dynamic_partition_pruning_2.q.out       |   7 ++
 .../tez/dynpart_sort_opt_vectorization.q.out    |  17 +++
 .../tez/dynpart_sort_optimization.q.out         |  15 +++
 .../tez/dynpart_sort_optimization2.q.out        |   8 ++
 .../tez/filter_join_breaktask.q.out             |   1 +
 .../results/clientpositive/tez/groupby1.q.out   |   1 +
 .../results/clientpositive/tez/groupby2.q.out   |   1 +
 .../results/clientpositive/tez/groupby3.q.out   |   1 +
 .../results/clientpositive/tez/having.q.out     |   7 +-
 .../tez/hybridgrace_hashjoin_1.q.out            |  12 ++
 .../tez/hybridgrace_hashjoin_2.q.out            |   8 ++
 .../clientpositive/tez/insert_into1.q.out       |   4 +
 .../clientpositive/tez/insert_into2.q.out       |   4 +
 .../clientpositive/tez/join0.q.java1.7.out      |   2 +-
 .../test/results/clientpositive/tez/join1.q.out |   1 +
 .../clientpositive/tez/join_nullsafe.q.out      |   6 +
 .../clientpositive/tez/limit_pushdown.q.out     |  14 ++-
 .../clientpositive/tez/llapdecider.q.out        |  16 ++-
 .../clientpositive/tez/load_dyn_part2.q.out     |   1 +
 .../clientpositive/tez/lvj_mapjoin.q.out        |   1 +
 .../clientpositive/tez/mapjoin_decimal.q.out    |   1 +
 .../clientpositive/tez/mapjoin_mapjoin.q.out    |   4 +-
 .../results/clientpositive/tez/mapreduce1.q.out |   1 +
 .../results/clientpositive/tez/mapreduce2.q.out |   1 +
 .../results/clientpositive/tez/merge1.q.out     |   1 +
 .../results/clientpositive/tez/merge2.q.out     |   1 +
 .../results/clientpositive/tez/mergejoin.q.out  |  13 +-
 .../tez/metadata_only_queries.q.out             |   5 +
 .../clientpositive/tez/metadataonly1.q.out      |   9 ++
 .../test/results/clientpositive/tez/mrr.q.out   |   6 +-
 .../clientpositive/tez/optimize_nullscan.q.out  |   7 ++
 .../results/clientpositive/tez/parallel.q.out   |   1 +
 .../test/results/clientpositive/tez/ptf.q.out   |  24 +++-
 .../clientpositive/tez/ptf_matchpath.q.out      |   3 +
 .../clientpositive/tez/ptf_streaming.q.out      |  13 +-
 .../clientpositive/tez/script_pipe.q.out        |   2 +-
 .../clientpositive/tez/selectDistinctStar.q.out |   7 +-
 .../results/clientpositive/tez/skewjoin.q.out   |   8 ++
 .../clientpositive/tez/stats_only_null.q.out    |   2 +
 .../clientpositive/tez/subquery_exists.q.out    |   2 +-
 .../clientpositive/tez/subquery_in.q.out        |   7 +-
 .../results/clientpositive/tez/temp_table.q.out |   2 +
 .../tez/tez_bmj_schema_evolution.q.out          |   1 +
 .../results/clientpositive/tez/tez_dml.q.out    |   2 +-
 .../tez/tez_dynpart_hashjoin_1.q.out            |   7 +-
 .../tez/tez_dynpart_hashjoin_2.q.out            |   4 +-
 .../tez/tez_dynpart_hashjoin_3.q.out            |   3 +-
 .../results/clientpositive/tez/tez_join.q.out   |   1 +
 .../clientpositive/tez/tez_join_hash.q.out      |   2 +
 .../tez/tez_join_result_complex.q.out           |   2 +
 .../clientpositive/tez/tez_join_tests.q.out     |   2 +-
 .../clientpositive/tez/tez_joins_explain.q.out  |   2 +-
 .../clientpositive/tez/tez_self_join.q.out      |   1 +
 .../results/clientpositive/tez/tez_smb_1.q.out  |   4 +
 .../clientpositive/tez/tez_smb_empty.q.out      |   5 +
 .../clientpositive/tez/tez_smb_main.q.out       |  12 +-
 .../results/clientpositive/tez/tez_union.q.out  |   8 +-
 .../results/clientpositive/tez/tez_union2.q.out |   2 +-
 .../tez/tez_union_dynamic_partition.q.out       |   1 +
 .../clientpositive/tez/tez_union_group_by.q.out |   1 +
 .../tez/tez_union_multiinsert.q.out             |   5 +
 .../tez/tez_vector_dynpart_hashjoin_1.q.out     |   7 +-
 .../tez/tez_vector_dynpart_hashjoin_2.q.out     |   4 +-
 .../clientpositive/tez/transform_ppr1.q.out     |   2 +-
 .../clientpositive/tez/transform_ppr2.q.out     |   2 +-
 .../results/clientpositive/tez/union3.q.out     |   2 +-
 .../results/clientpositive/tez/union4.q.out     |   1 +
 .../results/clientpositive/tez/union6.q.out     |   1 +
 .../clientpositive/tez/unionDistinct_1.q.out    |  43 +++++++
 .../clientpositive/tez/vector_aggregate_9.q.out |   1 +
 .../tez/vector_between_columns.q.out            |   1 +
 .../clientpositive/tez/vector_between_in.q.out  |   8 ++
 .../tez/vector_binary_join_groupby.q.out        |   2 +
 .../clientpositive/tez/vector_bucket.q.out      |   1 +
 .../tez/vector_cast_constant.q.java1.7.out      |   1 +
 .../clientpositive/tez/vector_char_2.q.out      |   2 +
 .../tez/vector_char_mapjoin1.q.out              |   3 +
 .../clientpositive/tez/vector_char_simple.q.out |   3 +
 .../clientpositive/tez/vector_coalesce.q.out    |   6 +-
 .../clientpositive/tez/vector_coalesce_2.q.out  |   2 +
 .../tez/vector_count_distinct.q.out             |   1 +
 .../clientpositive/tez/vector_data_types.q.out  |   2 +
 .../clientpositive/tez/vector_date_1.q.out      |   6 +
 .../clientpositive/tez/vector_decimal_1.q.out   |   9 ++
 .../tez/vector_decimal_10_0.q.out               |   1 +
 .../clientpositive/tez/vector_decimal_2.q.out   |  27 +++++
 .../tez/vector_decimal_aggregate.q.out          |   2 +
 .../tez/vector_decimal_expressions.q.out        |   1 +
 .../tez/vector_decimal_mapjoin.q.out            |   1 +
 .../tez/vector_decimal_precision.q.out          |   1 +
 .../tez/vector_decimal_round.q.out              |   6 +
 .../tez/vector_decimal_round_2.q.out            |   4 +
 .../clientpositive/tez/vector_decimal_udf.q.out |   7 ++
 .../clientpositive/tez/vector_distinct_2.q.out  |   1 +
 .../clientpositive/tez/vector_groupby_3.q.out   |   1 +
 .../tez/vector_groupby_reduce.q.out             |   4 +
 .../tez/vector_grouping_sets.q.out              |   2 +
 .../clientpositive/tez/vector_if_expr.q.out     |   2 +-
 .../clientpositive/tez/vector_inner_join.q.out  |   9 ++
 .../clientpositive/tez/vector_interval_1.q.out  |   8 ++
 .../clientpositive/tez/vector_interval_2.q.out  |  10 ++
 .../tez/vector_interval_mapjoin.q.out           |   1 +
 .../clientpositive/tez/vector_join30.q.out      |   8 ++
 .../tez/vector_left_outer_join.q.out            |   2 +-
 .../tez/vector_left_outer_join2.q.out           |   6 +
 .../tez/vector_leftsemi_mapjoin.q.out           | 120 +++++++++++++++++++
 .../tez/vector_mapjoin_reduce.q.out             |   3 +-
 .../tez/vector_mr_diff_schema_alias.q.out       |   1 +
 .../tez/vector_non_string_partition.q.out       |   2 +
 .../tez/vector_null_projection.q.out            |   1 +
 .../tez/vector_nullsafe_join.q.out              |  10 ++
 .../clientpositive/tez/vector_orderby_5.q.out   |   1 +
 .../clientpositive/tez/vector_outer_join0.q.out |   2 +
 .../clientpositive/tez/vector_outer_join1.q.out |   3 +
 .../clientpositive/tez/vector_outer_join2.q.out |   1 +
 .../clientpositive/tez/vector_outer_join3.q.out |   3 +
 .../clientpositive/tez/vector_outer_join4.q.out |   3 +
 .../clientpositive/tez/vector_outer_join5.q.out |  10 ++
 .../clientpositive/tez/vector_outer_join6.q.out |   2 +
 .../tez/vector_partition_diff_num_cols.q.out    |   5 +
 .../tez/vector_partitioned_date_time.q.out      |   6 +
 .../clientpositive/tez/vector_reduce1.q.out     |   1 +
 .../clientpositive/tez/vector_reduce2.q.out     |   1 +
 .../clientpositive/tez/vector_reduce3.q.out     |   1 +
 .../tez/vector_reduce_groupby_decimal.q.out     |   1 +
 .../tez/vector_string_concat.q.out              |   1 +
 .../tez/vector_varchar_mapjoin1.q.out           |   3 +
 .../tez/vector_varchar_simple.q.out             |   3 +
 .../tez/vector_when_case_null.q.out             |   1 +
 .../clientpositive/tez/vectorization_0.q.out    |  11 +-
 .../clientpositive/tez/vectorization_13.q.out   |   3 +-
 .../clientpositive/tez/vectorization_14.q.out   |   2 +-
 .../clientpositive/tez/vectorization_15.q.out   |   2 +-
 .../clientpositive/tez/vectorization_16.q.out   |   2 +-
 .../clientpositive/tez/vectorization_17.q.out   |   2 +-
 .../clientpositive/tez/vectorization_7.q.out    |   3 +-
 .../clientpositive/tez/vectorization_8.q.out    |   3 +-
 .../clientpositive/tez/vectorization_9.q.out    |   2 +-
 .../clientpositive/tez/vectorization_div0.q.out |   2 +
 .../tez/vectorization_limit.q.out               |   5 +
 .../tez/vectorization_part_project.q.out        |   1 +
 .../tez/vectorization_pushdown.q.out            |   2 +-
 .../tez/vectorization_short_regress.q.out       |  21 +++-
 .../tez/vectorized_bucketmapjoin1.q.out         |   3 +
 .../clientpositive/tez/vectorized_context.q.out |   1 +
 .../tez/vectorized_date_funcs.q.out             |   1 +
 .../tez/vectorized_distinct_gby.q.out           |   2 +
 .../vectorized_dynamic_partition_pruning.q.out  |  39 ++++++
 .../clientpositive/tez/vectorized_mapjoin.q.out |   2 +-
 .../tez/vectorized_nested_mapjoin.q.out         |   2 +-
 .../clientpositive/tez/vectorized_parquet.q.out |   1 +
 .../clientpositive/tez/vectorized_ptf.q.out     |  23 ++++
 .../tez/vectorized_shufflejoin.q.out            |   2 +-
 .../tez/vectorized_timestamp_funcs.q.out        |   7 ++
 226 files changed, 1253 insertions(+), 80 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/cbfd2570/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
index b2866e3..61a4cdc 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
@@ -1447,6 +1447,7 @@ public class QTestUtil {
       ".*USING 'java -cp.*",
       "^Deleted.*",
       ".*DagName:.*",
+      ".*DagId:.*",
       ".*Input:.*/data/files/.*",
       ".*Output:.*/data/files/.*",
       ".*total number of created files now is.*",

http://git-wip-us.apache.org/repos/asf/hive/blob/cbfd2570/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
index 28532c2..3d24464 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
@@ -641,7 +641,7 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
     DriverContext driverCxt = new DriverContext();
     Task task;
     if (conf.getVar(ConfVars.HIVE_EXECUTION_ENGINE).equals("tez")) {
-      TezWork tezWork = new TezWork(conf.getVar(HiveConf.ConfVars.HIVEQUERYID));
+      TezWork tezWork = new TezWork(conf.getVar(HiveConf.ConfVars.HIVEQUERYID), conf);
       mergeWork.setName("File Merge");
       tezWork.add(mergeWork);
       task = new TezTask();

http://git-wip-us.apache.org/repos/asf/hive/blob/cbfd2570/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DagUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DagUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DagUtils.java
index 63fb37a..cf031fe 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DagUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DagUtils.java
@@ -1228,12 +1228,7 @@ public class DagUtils {
   }
 
   public String createDagName(Configuration conf, QueryPlan plan) {
-    String name = HiveConf.getVar(conf, HiveConf.ConfVars.HIVEQUERYNAME);
-
-    if (name == null) {
-      name = conf.get("mapred.job.name");
-    }
-
+    String name = getUserSpecifiedDagName(conf);
     if (name == null) {
       name = plan.getQueryId();
     }
@@ -1242,7 +1237,13 @@ public class DagUtils {
     return name;
   }
 
+  public static String getUserSpecifiedDagName(Configuration conf) {
+    String name = HiveConf.getVar(conf, HiveConf.ConfVars.HIVEQUERYNAME);
+    return (name != null) ? name : conf.get("mapred.job.name");
+  }
+
   private DagUtils() {
     // don't instantiate
   }
+
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/cbfd2570/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java
index 2c3836b..1e93abe 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java
@@ -1332,7 +1332,7 @@ public final class GenMapRedUtils {
       cplan = GenMapRedUtils.createMergeTask(fsInputDesc, finalName,
           dpCtx != null && dpCtx.getNumDPCols() > 0, fsInput.getCompilationOpContext());
       if (conf.getVar(ConfVars.HIVE_EXECUTION_ENGINE).equals("tez")) {
-        work = new TezWork(conf.getVar(HiveConf.ConfVars.HIVEQUERYID));
+        work = new TezWork(conf.getVar(HiveConf.ConfVars.HIVEQUERYID), conf);
         cplan.setName("File Merge");
         ((TezWork) work).add(cplan);
       } else if (conf.getVar(ConfVars.HIVE_EXECUTION_ENGINE).equals("spark")) {
@@ -1345,7 +1345,7 @@ public final class GenMapRedUtils {
     } else {
       cplan = createMRWorkForMergingFiles(conf, tsMerge, fsInputDesc);
       if (conf.getVar(ConfVars.HIVE_EXECUTION_ENGINE).equals("tez")) {
-        work = new TezWork(conf.getVar(HiveConf.ConfVars.HIVEQUERYID));
+        work = new TezWork(conf.getVar(HiveConf.ConfVars.HIVEQUERYID), conf);
         cplan.setName("File Merge");
         ((TezWork)work).add(cplan);
       } else if (conf.getVar(ConfVars.HIVE_EXECUTION_ENGINE).equals("spark")) {

http://git-wip-us.apache.org/repos/asf/hive/blob/cbfd2570/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezProcContext.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezProcContext.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezProcContext.java
index 9334c73..ec5ef0e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezProcContext.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezProcContext.java
@@ -173,7 +173,7 @@ public class GenTezProcContext implements NodeProcessorCtx{
     this.inputs = inputs;
     this.outputs = outputs;
     this.currentTask = (TezTask) TaskFactory.get(
-         new TezWork(conf.getVar(HiveConf.ConfVars.HIVEQUERYID)), conf);
+         new TezWork(conf.getVar(HiveConf.ConfVars.HIVEQUERYID), conf), conf);
     this.leafOperatorToFollowingWork = new LinkedHashMap<Operator<?>, BaseWork>();
     this.linkOpWithWorkMap = new LinkedHashMap<Operator<?>, Map<BaseWork, TezEdgeProperty>>();
     this.linkWorkWithReduceSinkMap = new LinkedHashMap<BaseWork, List<ReduceSinkOperator>>();

http://git-wip-us.apache.org/repos/asf/hive/blob/cbfd2570/ql/src/java/org/apache/hadoop/hive/ql/plan/TezWork.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/TezWork.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/TezWork.java
index 8b82c66..e345215 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/TezWork.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/TezWork.java
@@ -34,6 +34,8 @@ import org.apache.commons.lang3.tuple.ImmutablePair;
 import org.apache.commons.lang3.tuple.Pair;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.exec.tez.DagUtils;
 import org.apache.hadoop.hive.ql.plan.TezEdgeProperty.EdgeType;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.hive.ql.plan.Explain.Level;
@@ -68,7 +70,8 @@ public class TezWork extends AbstractOperatorDesc {
   private static transient final Logger LOG = LoggerFactory.getLogger(TezWork.class);
 
   private static int counter;
-  private final String name;
+  private final String dagId;
+  private final String queryName;
   private final Set<BaseWork> roots = new HashSet<BaseWork>();
   private final Set<BaseWork> leaves = new HashSet<BaseWork>();
   private final Map<BaseWork, List<BaseWork>> workGraph = new HashMap<BaseWork, List<BaseWork>>();
@@ -77,13 +80,19 @@ public class TezWork extends AbstractOperatorDesc {
       new HashMap<Pair<BaseWork, BaseWork>, TezEdgeProperty>();
   private final Map<BaseWork, VertexType> workVertexTypeMap = new HashMap<BaseWork, VertexType>();
 
-  public TezWork(String name) {
-    this.name = name + ":" + (++counter);
+  public TezWork(String queryId, Configuration conf) {
+    this.dagId = queryId + ":" + (++counter);
+    this.queryName = (conf != null) ? DagUtils.getUserSpecifiedDagName(conf) : null;
   }
 
   @Explain(displayName = "DagName")
   public String getName() {
-    return name;
+    return queryName;
+  }
+
+  @Explain(displayName = "DagId")
+  public String getDagId() {
+    return dagId;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/cbfd2570/ql/src/test/org/apache/hadoop/hive/ql/exec/tez/TestTezTask.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/tez/TestTezTask.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/tez/TestTezTask.java
index 858cca0..dc39cbd 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/tez/TestTezTask.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/tez/TestTezTask.java
@@ -117,7 +117,7 @@ public class TestTezTask {
           }
         });
 
-    work = new TezWork("");
+    work = new TezWork("", null);
 
     mws = new MapWork[] { new MapWork(), new MapWork()};
     rws = new ReduceWork[] { new ReduceWork(), new ReduceWork() };
@@ -203,7 +203,7 @@ public class TestTezTask {
 
   @Test
   public void testEmptyWork() throws IllegalArgumentException, IOException, Exception {
-    DAG dag = task.build(conf, new TezWork(""), path, appLr, null, new Context(conf));
+    DAG dag = task.build(conf, new TezWork("", null), path, appLr, null, new Context(conf));
     assertEquals(dag.getVertices().size(), 0);
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/cbfd2570/ql/src/test/org/apache/hadoop/hive/ql/plan/TestTezWork.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/plan/TestTezWork.java b/ql/src/test/org/apache/hadoop/hive/ql/plan/TestTezWork.java
index 2344279..c1e192c 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/plan/TestTezWork.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/plan/TestTezWork.java
@@ -39,7 +39,7 @@ public class TestTezWork {
   @Before
   public void setup() throws Exception {
     nodes = new LinkedList<BaseWork>();
-    work = new TezWork("");
+    work = new TezWork("", null);
     addWork(5);
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/cbfd2570/ql/src/test/results/clientpositive/llap/bucket_map_join_tez1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/bucket_map_join_tez1.q.out b/ql/src/test/results/clientpositive/llap/bucket_map_join_tez1.q.out
index 02d4fdd..46638e9 100644
--- a/ql/src/test/results/clientpositive/llap/bucket_map_join_tez1.q.out
+++ b/ql/src/test/results/clientpositive/llap/bucket_map_join_tez1.q.out
@@ -123,6 +123,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 2 <- Map 1 (CUSTOM_EDGE)
 #### A masked pattern was here ####
@@ -205,6 +206,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (BROADCAST_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
@@ -340,6 +342,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 3 (CUSTOM_SIMPLE_EDGE)
 #### A masked pattern was here ####
@@ -447,6 +450,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 3 (CUSTOM_EDGE)
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (CUSTOM_SIMPLE_EDGE)
@@ -589,6 +593,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 3 (CUSTOM_EDGE)
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (CUSTOM_SIMPLE_EDGE)
@@ -730,6 +735,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 2 (CUSTOM_EDGE), Map 3 (CUSTOM_EDGE)
 #### A masked pattern was here ####
@@ -831,6 +837,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 2 (CUSTOM_EDGE), Map 3 (CUSTOM_EDGE)
 #### A masked pattern was here ####
@@ -944,6 +951,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 3 (CUSTOM_SIMPLE_EDGE)
 #### A masked pattern was here ####
@@ -1047,6 +1055,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 3 (CUSTOM_SIMPLE_EDGE)
 #### A masked pattern was here ####
@@ -1144,6 +1153,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 2 <- Map 1 (BROADCAST_EDGE)
 #### A masked pattern was here ####
@@ -1242,6 +1252,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 2 <- Map 1 (CUSTOM_EDGE)
 #### A masked pattern was here ####
@@ -1318,6 +1329,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 2 (CUSTOM_EDGE), Map 3 (BROADCAST_EDGE)
 #### A masked pattern was here ####
@@ -1427,6 +1439,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 2 <- Map 1 (BROADCAST_EDGE)
 #### A masked pattern was here ####
@@ -1536,6 +1549,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 3 (CUSTOM_EDGE)
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
@@ -1652,6 +1666,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 3 (CUSTOM_EDGE)
         Reducer 2 <- Map 1 (SIMPLE_EDGE)

http://git-wip-us.apache.org/repos/asf/hive/blob/cbfd2570/ql/src/test/results/clientpositive/llap/bucket_map_join_tez2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/bucket_map_join_tez2.q.out b/ql/src/test/results/clientpositive/llap/bucket_map_join_tez2.q.out
index 5e7af65..833cd96 100644
--- a/ql/src/test/results/clientpositive/llap/bucket_map_join_tez2.q.out
+++ b/ql/src/test/results/clientpositive/llap/bucket_map_join_tez2.q.out
@@ -119,6 +119,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 2 (CUSTOM_EDGE), Map 3 (BROADCAST_EDGE)
 #### A masked pattern was here ####
@@ -250,6 +251,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 2 <- Map 1 (BROADCAST_EDGE)
 #### A masked pattern was here ####
@@ -328,6 +330,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 2 (CUSTOM_EDGE)
 #### A masked pattern was here ####
@@ -401,6 +404,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 2 (CUSTOM_EDGE)
 #### A masked pattern was here ####
@@ -474,6 +478,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 2 <- Map 1 (CUSTOM_EDGE)
 #### A masked pattern was here ####
@@ -545,6 +550,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 3 (CUSTOM_SIMPLE_EDGE)
 #### A masked pattern was here ####
@@ -630,6 +636,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 3 (BROADCAST_EDGE)
 #### A masked pattern was here ####

http://git-wip-us.apache.org/repos/asf/hive/blob/cbfd2570/ql/src/test/results/clientpositive/llap/dynamic_partition_pruning.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/dynamic_partition_pruning.q.out b/ql/src/test/results/clientpositive/llap/dynamic_partition_pruning.q.out
index d2c11d7..d525bd9 100644
--- a/ql/src/test/results/clientpositive/llap/dynamic_partition_pruning.q.out
+++ b/ql/src/test/results/clientpositive/llap/dynamic_partition_pruning.q.out
@@ -48,6 +48,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
@@ -199,6 +200,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
@@ -325,6 +327,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
@@ -451,6 +454,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
@@ -577,6 +581,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
@@ -692,6 +697,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
         Reducer 3 <- Map 6 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
@@ -875,6 +881,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
         Reducer 3 <- Map 6 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
@@ -1040,6 +1047,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
@@ -1180,6 +1188,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
@@ -1304,6 +1313,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
@@ -1430,6 +1440,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
@@ -1552,6 +1563,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
@@ -1678,6 +1690,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
@@ -1804,6 +1817,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
@@ -1915,6 +1929,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
@@ -2039,6 +2054,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
@@ -2180,6 +2196,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
@@ -2336,6 +2353,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
@@ -2453,6 +2471,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
@@ -2595,6 +2614,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
@@ -2702,6 +2722,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
@@ -2810,6 +2831,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
@@ -2920,6 +2942,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
         Reducer 3 <- Map 6 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
@@ -3080,6 +3103,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
         Reducer 3 <- Map 6 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
@@ -3225,6 +3249,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Union 6 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
@@ -3429,6 +3454,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Union 6 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
@@ -3636,6 +3662,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 11 <- Map 10 (SIMPLE_EDGE), Union 9 (CONTAINS)
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Union 3 (CONTAINS)
@@ -3906,6 +3933,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 3 (BROADCAST_EDGE)
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
@@ -4042,6 +4070,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 3 (BROADCAST_EDGE)
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
@@ -4167,6 +4196,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 3 (BROADCAST_EDGE), Map 4 (BROADCAST_EDGE)
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
@@ -4349,6 +4379,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 3 (BROADCAST_EDGE)
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
@@ -4497,6 +4528,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 3 (BROADCAST_EDGE)
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
@@ -4607,6 +4639,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 3 (BROADCAST_EDGE)
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
@@ -4728,6 +4761,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 3 (BROADCAST_EDGE)
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
@@ -4864,6 +4898,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 3 <- Map 1 (BROADCAST_EDGE), Map 2 (SIMPLE_EDGE)
         Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
@@ -4998,6 +5033,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 3 (BROADCAST_EDGE)
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
@@ -5100,6 +5136,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 3 (BROADCAST_EDGE)
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
@@ -5188,6 +5225,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 2 <- Map 1 (BROADCAST_EDGE)
         Reducer 3 <- Map 2 (SIMPLE_EDGE)
@@ -5278,6 +5316,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 3 (BROADCAST_EDGE), Map 4 (BROADCAST_EDGE)
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
@@ -5427,6 +5466,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 2 <- Map 1 (BROADCAST_EDGE), Map 4 (BROADCAST_EDGE)
         Reducer 3 <- Map 2 (SIMPLE_EDGE)
@@ -5554,6 +5594,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Union 5 (BROADCAST_EDGE)
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
@@ -5793,6 +5834,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 3 (BROADCAST_EDGE)
         Reducer 2 <- Map 1 (SIMPLE_EDGE)

http://git-wip-us.apache.org/repos/asf/hive/blob/cbfd2570/ql/src/test/results/clientpositive/llap/dynamic_partition_pruning_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/dynamic_partition_pruning_2.q.out b/ql/src/test/results/clientpositive/llap/dynamic_partition_pruning_2.q.out
index 49a1efc..86d0c1c 100644
--- a/ql/src/test/results/clientpositive/llap/dynamic_partition_pruning_2.q.out
+++ b/ql/src/test/results/clientpositive/llap/dynamic_partition_pruning_2.q.out
@@ -154,6 +154,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 4 (BROADCAST_EDGE)
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
@@ -325,6 +326,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 4 (BROADCAST_EDGE)
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
@@ -473,6 +475,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 2 (BROADCAST_EDGE)
 #### A masked pattern was here ####
@@ -587,6 +590,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 2 (BROADCAST_EDGE)
 #### A masked pattern was here ####
@@ -692,6 +696,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 4 (BROADCAST_EDGE)
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
@@ -855,6 +860,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 3 (BROADCAST_EDGE), Union 2 (CONTAINS)
         Map 4 <- Map 5 (BROADCAST_EDGE), Union 2 (CONTAINS)
@@ -1044,6 +1050,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 3 (BROADCAST_EDGE)
         Reducer 2 <- Map 1 (SIMPLE_EDGE)

http://git-wip-us.apache.org/repos/asf/hive/blob/cbfd2570/ql/src/test/results/clientpositive/llap/hybridgrace_hashjoin_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/hybridgrace_hashjoin_1.q.out b/ql/src/test/results/clientpositive/llap/hybridgrace_hashjoin_1.q.out
index dd7fb01..e2598fd 100644
--- a/ql/src/test/results/clientpositive/llap/hybridgrace_hashjoin_1.q.out
+++ b/ql/src/test/results/clientpositive/llap/hybridgrace_hashjoin_1.q.out
@@ -44,6 +44,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 3 (BROADCAST_EDGE)
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
@@ -166,6 +167,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 3 (BROADCAST_EDGE)
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
@@ -285,6 +287,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 3 (BROADCAST_EDGE)
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
@@ -403,6 +406,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 3 (BROADCAST_EDGE)
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
@@ -520,6 +524,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 3 (BROADCAST_EDGE)
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
@@ -630,6 +635,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 3 (BROADCAST_EDGE)
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
@@ -775,6 +781,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 3 (BROADCAST_EDGE)
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
@@ -895,6 +902,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 3 (BROADCAST_EDGE)
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
@@ -1016,6 +1024,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 3 (BROADCAST_EDGE)
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
@@ -1136,6 +1145,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 3 (BROADCAST_EDGE)
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
@@ -1281,6 +1291,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 2 (BROADCAST_EDGE)
 #### A masked pattern was here ####
@@ -1476,6 +1487,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 2 (BROADCAST_EDGE)
 #### A masked pattern was here ####

http://git-wip-us.apache.org/repos/asf/hive/blob/cbfd2570/ql/src/test/results/clientpositive/llap/hybridgrace_hashjoin_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/hybridgrace_hashjoin_2.q.out b/ql/src/test/results/clientpositive/llap/hybridgrace_hashjoin_2.q.out
index d721215..ba23620 100644
--- a/ql/src/test/results/clientpositive/llap/hybridgrace_hashjoin_2.q.out
+++ b/ql/src/test/results/clientpositive/llap/hybridgrace_hashjoin_2.q.out
@@ -39,6 +39,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 2 <- Map 1 (BROADCAST_EDGE), Map 4 (BROADCAST_EDGE)
         Reducer 3 <- Map 2 (SIMPLE_EDGE)
@@ -166,6 +167,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 2 <- Map 1 (BROADCAST_EDGE), Map 4 (BROADCAST_EDGE)
         Reducer 3 <- Map 2 (SIMPLE_EDGE)
@@ -307,6 +309,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 2 <- Map 1 (BROADCAST_EDGE), Map 4 (BROADCAST_EDGE), Map 5 (BROADCAST_EDGE)
         Reducer 3 <- Map 2 (SIMPLE_EDGE)
@@ -455,6 +458,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 2 <- Map 1 (BROADCAST_EDGE), Map 4 (BROADCAST_EDGE), Map 5 (BROADCAST_EDGE)
         Reducer 3 <- Map 2 (SIMPLE_EDGE)
@@ -621,6 +625,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 2 <- Map 1 (BROADCAST_EDGE), Map 6 (BROADCAST_EDGE)
         Map 8 <- Map 10 (BROADCAST_EDGE), Map 7 (BROADCAST_EDGE)
@@ -864,6 +869,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 2 <- Map 1 (BROADCAST_EDGE), Map 6 (BROADCAST_EDGE)
         Map 8 <- Map 10 (BROADCAST_EDGE), Map 7 (BROADCAST_EDGE)
@@ -1122,6 +1128,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 2 <- Map 1 (BROADCAST_EDGE), Map 4 (BROADCAST_EDGE), Map 5 (BROADCAST_EDGE), Map 6 (BROADCAST_EDGE)
         Reducer 3 <- Map 2 (SIMPLE_EDGE)
@@ -1311,6 +1318,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 2 <- Map 1 (BROADCAST_EDGE), Map 4 (BROADCAST_EDGE), Map 5 (BROADCAST_EDGE), Map 6 (BROADCAST_EDGE)
         Reducer 3 <- Map 2 (SIMPLE_EDGE)

http://git-wip-us.apache.org/repos/asf/hive/blob/cbfd2570/ql/src/test/results/clientpositive/llap/llapdecider.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/llapdecider.q.out b/ql/src/test/results/clientpositive/llap/llapdecider.q.out
index fd33181..5879bf6 100644
--- a/ql/src/test/results/clientpositive/llap/llapdecider.q.out
+++ b/ql/src/test/results/clientpositive/llap/llapdecider.q.out
@@ -11,10 +11,10 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
-#### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
@@ -91,6 +91,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
@@ -161,6 +162,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
@@ -242,6 +244,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
@@ -315,6 +318,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
@@ -403,6 +407,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
@@ -494,6 +499,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
@@ -584,6 +590,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
@@ -671,6 +678,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
@@ -758,6 +766,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
@@ -847,6 +856,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
@@ -934,6 +944,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
@@ -1031,6 +1042,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
@@ -1090,6 +1102,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
@@ -1147,6 +1160,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####

http://git-wip-us.apache.org/repos/asf/hive/blob/cbfd2570/ql/src/test/results/clientpositive/llap/lvj_mapjoin.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/lvj_mapjoin.q.out b/ql/src/test/results/clientpositive/llap/lvj_mapjoin.q.out
index 182dd94..99c7a38 100644
--- a/ql/src/test/results/clientpositive/llap/lvj_mapjoin.q.out
+++ b/ql/src/test/results/clientpositive/llap/lvj_mapjoin.q.out
@@ -109,6 +109,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 2 (BROADCAST_EDGE)
 #### A masked pattern was here ####

http://git-wip-us.apache.org/repos/asf/hive/blob/cbfd2570/ql/src/test/results/clientpositive/llap/mapjoin_decimal.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/mapjoin_decimal.q.out b/ql/src/test/results/clientpositive/llap/mapjoin_decimal.q.out
index 9fac69f..e49ba0e 100644
--- a/ql/src/test/results/clientpositive/llap/mapjoin_decimal.q.out
+++ b/ql/src/test/results/clientpositive/llap/mapjoin_decimal.q.out
@@ -89,6 +89,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 3 (BROADCAST_EDGE)
         Reducer 2 <- Map 1 (SIMPLE_EDGE)

http://git-wip-us.apache.org/repos/asf/hive/blob/cbfd2570/ql/src/test/results/clientpositive/llap/mrr.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/mrr.q.out b/ql/src/test/results/clientpositive/llap/mrr.q.out
index b5ef1f6..ddd4f7f 100644
--- a/ql/src/test/results/clientpositive/llap/mrr.q.out
+++ b/ql/src/test/results/clientpositive/llap/mrr.q.out
@@ -15,10 +15,10 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
-#### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
@@ -407,6 +407,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
@@ -840,6 +841,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 4 (BROADCAST_EDGE)
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
@@ -1296,6 +1298,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 10 <- Map 9 (SIMPLE_EDGE)
         Reducer 11 <- Reducer 10 (SIMPLE_EDGE)
@@ -1688,6 +1691,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (BROADCAST_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)

http://git-wip-us.apache.org/repos/asf/hive/blob/cbfd2570/ql/src/test/results/clientpositive/llap/tez_bmj_schema_evolution.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/tez_bmj_schema_evolution.q.out b/ql/src/test/results/clientpositive/llap/tez_bmj_schema_evolution.q.out
index 73b89c3..7b48f27 100644
--- a/ql/src/test/results/clientpositive/llap/tez_bmj_schema_evolution.q.out
+++ b/ql/src/test/results/clientpositive/llap/tez_bmj_schema_evolution.q.out
@@ -77,6 +77,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 3 (CUSTOM_EDGE)
         Reducer 2 <- Map 1 (SIMPLE_EDGE)

http://git-wip-us.apache.org/repos/asf/hive/blob/cbfd2570/ql/src/test/results/clientpositive/llap/tez_dml.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/tez_dml.q.out b/ql/src/test/results/clientpositive/llap/tez_dml.q.out
index a87ccf3..352824a 100644
--- a/ql/src/test/results/clientpositive/llap/tez_dml.q.out
+++ b/ql/src/test/results/clientpositive/llap/tez_dml.q.out
@@ -18,10 +18,10 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
-#### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:

http://git-wip-us.apache.org/repos/asf/hive/blob/cbfd2570/ql/src/test/results/clientpositive/llap/tez_dynpart_hashjoin_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/tez_dynpart_hashjoin_1.q.out b/ql/src/test/results/clientpositive/llap/tez_dynpart_hashjoin_1.q.out
index b459692..5d7f2ef 100644
--- a/ql/src/test/results/clientpositive/llap/tez_dynpart_hashjoin_1.q.out
+++ b/ql/src/test/results/clientpositive/llap/tez_dynpart_hashjoin_1.q.out
@@ -23,10 +23,10 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
-#### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
@@ -152,6 +152,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
@@ -276,6 +277,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
@@ -427,6 +429,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Map 4 (CUSTOM_SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
@@ -559,6 +562,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Map 4 (CUSTOM_SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
@@ -686,6 +690,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Map 5 (CUSTOM_SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)

http://git-wip-us.apache.org/repos/asf/hive/blob/cbfd2570/ql/src/test/results/clientpositive/llap/tez_dynpart_hashjoin_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/tez_dynpart_hashjoin_2.q.out b/ql/src/test/results/clientpositive/llap/tez_dynpart_hashjoin_2.q.out
index 56f0165..e975ffc 100644
--- a/ql/src/test/results/clientpositive/llap/tez_dynpart_hashjoin_2.q.out
+++ b/ql/src/test/results/clientpositive/llap/tez_dynpart_hashjoin_2.q.out
@@ -33,10 +33,10 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
-#### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
@@ -224,6 +224,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
@@ -413,6 +414,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)

http://git-wip-us.apache.org/repos/asf/hive/blob/cbfd2570/ql/src/test/results/clientpositive/llap/tez_join.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/tez_join.q.out b/ql/src/test/results/clientpositive/llap/tez_join.q.out
index 41e2156..dc32911 100644
--- a/ql/src/test/results/clientpositive/llap/tez_join.q.out
+++ b/ql/src/test/results/clientpositive/llap/tez_join.q.out
@@ -47,6 +47,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 3 (SIMPLE_EDGE)
 #### A masked pattern was here ####

http://git-wip-us.apache.org/repos/asf/hive/blob/cbfd2570/ql/src/test/results/clientpositive/llap/tez_join_hash.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/tez_join_hash.q.out b/ql/src/test/results/clientpositive/llap/tez_join_hash.q.out
index 366fedc..adacc33 100644
--- a/ql/src/test/results/clientpositive/llap/tez_join_hash.q.out
+++ b/ql/src/test/results/clientpositive/llap/tez_join_hash.q.out
@@ -33,6 +33,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
@@ -145,6 +146,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 7 <- Union 3 (CONTAINS)
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 6 (SIMPLE_EDGE), Union 3 (CONTAINS)

http://git-wip-us.apache.org/repos/asf/hive/blob/cbfd2570/ql/src/test/results/clientpositive/llap/tez_join_result_complex.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/tez_join_result_complex.q.out b/ql/src/test/results/clientpositive/llap/tez_join_result_complex.q.out
index 8f678b3..1a7e831 100644
--- a/ql/src/test/results/clientpositive/llap/tez_join_result_complex.q.out
+++ b/ql/src/test/results/clientpositive/llap/tez_join_result_complex.q.out
@@ -423,6 +423,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 2 <- Map 1 (BROADCAST_EDGE)
 #### A masked pattern was here ####
@@ -1644,6 +1645,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 2 <- Map 1 (BROADCAST_EDGE)
 #### A masked pattern was here ####

http://git-wip-us.apache.org/repos/asf/hive/blob/cbfd2570/ql/src/test/results/clientpositive/llap/tez_join_tests.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/tez_join_tests.q.out b/ql/src/test/results/clientpositive/llap/tez_join_tests.q.out
index cebe50f..aadbd31 100644
--- a/ql/src/test/results/clientpositive/llap/tez_join_tests.q.out
+++ b/ql/src/test/results/clientpositive/llap/tez_join_tests.q.out
@@ -15,12 +15,12 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 6 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
         Reducer 4 <- Map 7 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
         Reducer 5 <- Reducer 4 (SIMPLE_EDGE)
-#### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:

http://git-wip-us.apache.org/repos/asf/hive/blob/cbfd2570/ql/src/test/results/clientpositive/llap/tez_joins_explain.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/tez_joins_explain.q.out b/ql/src/test/results/clientpositive/llap/tez_joins_explain.q.out
index 2c61e23..0922fbe 100644
--- a/ql/src/test/results/clientpositive/llap/tez_joins_explain.q.out
+++ b/ql/src/test/results/clientpositive/llap/tez_joins_explain.q.out
@@ -13,12 +13,12 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 6 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
         Reducer 4 <- Map 7 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
         Reducer 5 <- Reducer 4 (SIMPLE_EDGE)
-#### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:

http://git-wip-us.apache.org/repos/asf/hive/blob/cbfd2570/ql/src/test/results/clientpositive/llap/tez_self_join.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/tez_self_join.q.out b/ql/src/test/results/clientpositive/llap/tez_self_join.q.out
index 9f1323f..0fe84b6 100644
--- a/ql/src/test/results/clientpositive/llap/tez_self_join.q.out
+++ b/ql/src/test/results/clientpositive/llap/tez_self_join.q.out
@@ -71,6 +71,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
         Reducer 3 <- Map 5 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)

http://git-wip-us.apache.org/repos/asf/hive/blob/cbfd2570/ql/src/test/results/clientpositive/llap/tez_smb_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/tez_smb_1.q.out b/ql/src/test/results/clientpositive/llap/tez_smb_1.q.out
index 747bfe7..8f13b56 100644
--- a/ql/src/test/results/clientpositive/llap/tez_smb_1.q.out
+++ b/ql/src/test/results/clientpositive/llap/tez_smb_1.q.out
@@ -121,6 +121,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
@@ -210,6 +211,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
@@ -352,6 +354,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
         Reducer 3 <- Map 5 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
@@ -496,6 +499,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 4 <- Map 1 (SIMPLE_EDGE), Map 3 (SIMPLE_EDGE)
         Reducer 5 <- Reducer 4 (SIMPLE_EDGE)

http://git-wip-us.apache.org/repos/asf/hive/blob/cbfd2570/ql/src/test/results/clientpositive/llap/tez_smb_main.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/tez_smb_main.q.out b/ql/src/test/results/clientpositive/llap/tez_smb_main.q.out
index f15339b..65458cf 100644
--- a/ql/src/test/results/clientpositive/llap/tez_smb_main.q.out
+++ b/ql/src/test/results/clientpositive/llap/tez_smb_main.q.out
@@ -11,9 +11,9 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 3 (SIMPLE_EDGE)
-#### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
@@ -250,6 +250,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 2 <- Map 1 (CUSTOM_EDGE)
         Reducer 3 <- Map 2 (SIMPLE_EDGE)
@@ -359,6 +360,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 2 <- Map 1 (CUSTOM_EDGE)
         Reducer 3 <- Map 2 (SIMPLE_EDGE)
@@ -468,6 +470,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 2 <- Map 1 (CUSTOM_EDGE)
         Reducer 3 <- Map 2 (SIMPLE_EDGE)
@@ -573,6 +576,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 2 <- Map 4 (BROADCAST_EDGE)
         Reducer 3 <- Map 2 (SIMPLE_EDGE)
@@ -697,6 +701,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
@@ -813,6 +818,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Union 2 (CONTAINS)
         Map 6 <- Union 2 (CONTAINS)
@@ -947,6 +953,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 2 <- Map 1 (BROADCAST_EDGE)
         Reducer 3 <- Map 2 (SIMPLE_EDGE)
@@ -1050,6 +1057,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 2 <- Map 1 (CUSTOM_EDGE), Map 4 (BROADCAST_EDGE)
         Reducer 3 <- Map 2 (SIMPLE_EDGE)
@@ -1193,6 +1201,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 4 (CUSTOM_EDGE), Map 6 (BROADCAST_EDGE), Union 2 (CONTAINS)
         Map 5 <- Map 6 (BROADCAST_EDGE), Union 2 (CONTAINS)
@@ -1364,6 +1373,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
         Reducer 4 <- Map 3 (SIMPLE_EDGE), Reducer 2 (CUSTOM_SIMPLE_EDGE)

http://git-wip-us.apache.org/repos/asf/hive/blob/cbfd2570/ql/src/test/results/clientpositive/llap/tez_union.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/tez_union.q.out b/ql/src/test/results/clientpositive/llap/tez_union.q.out
index 3bfa24c..f5cd30f 100644
--- a/ql/src/test/results/clientpositive/llap/tez_union.q.out
+++ b/ql/src/test/results/clientpositive/llap/tez_union.q.out
@@ -15,10 +15,10 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 3 (BROADCAST_EDGE), Union 2 (CONTAINS)
         Map 4 <- Union 2 (CONTAINS)
-#### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
@@ -164,6 +164,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Union 2 (CONTAINS)
         Map 5 <- Union 2 (CONTAINS)
@@ -338,6 +339,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 2 <- Map 1 (BROADCAST_EDGE), Union 3 (CONTAINS)
         Map 4 <- Map 1 (BROADCAST_EDGE), Union 3 (CONTAINS)
@@ -512,6 +514,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 10 (BROADCAST_EDGE), Map 4 (BROADCAST_EDGE), Map 9 (BROADCAST_EDGE), Union 2 (CONTAINS)
         Map 5 <- Map 10 (BROADCAST_EDGE), Map 6 (BROADCAST_EDGE), Map 9 (BROADCAST_EDGE), Union 2 (CONTAINS)
@@ -948,6 +951,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 3 (BROADCAST_EDGE), Union 2 (CONTAINS)
         Map 4 <- Map 5 (BROADCAST_EDGE), Union 2 (CONTAINS)
@@ -1137,6 +1141,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Map 4 (BROADCAST_EDGE), Union 2 (CONTAINS)
         Map 3 <- Map 4 (BROADCAST_EDGE), Union 2 (CONTAINS)
@@ -1246,6 +1251,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Union 2 (CONTAINS)
         Map 3 <- Union 2 (CONTAINS)

http://git-wip-us.apache.org/repos/asf/hive/blob/cbfd2570/ql/src/test/results/clientpositive/llap/tez_union2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/tez_union2.q.out b/ql/src/test/results/clientpositive/llap/tez_union2.q.out
index e7ea2cd..ef43308 100644
--- a/ql/src/test/results/clientpositive/llap/tez_union2.q.out
+++ b/ql/src/test/results/clientpositive/llap/tez_union2.q.out
@@ -39,13 +39,13 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Union 2 (CONTAINS)
         Map 4 <- Union 5 (CONTAINS)
         Map 7 <- Union 5 (CONTAINS)
         Reducer 3 <- Union 2 (SIMPLE_EDGE)
         Reducer 6 <- Union 2 (CONTAINS), Union 5 (SIMPLE_EDGE)
-#### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:

http://git-wip-us.apache.org/repos/asf/hive/blob/cbfd2570/ql/src/test/results/clientpositive/llap/tez_union_dynamic_partition.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/tez_union_dynamic_partition.q.out b/ql/src/test/results/clientpositive/llap/tez_union_dynamic_partition.q.out
index 3a68415..a9ef497 100644
--- a/ql/src/test/results/clientpositive/llap/tez_union_dynamic_partition.q.out
+++ b/ql/src/test/results/clientpositive/llap/tez_union_dynamic_partition.q.out
@@ -54,6 +54,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Union 2 (CONTAINS)
         Map 3 <- Union 2 (CONTAINS)

http://git-wip-us.apache.org/repos/asf/hive/blob/cbfd2570/ql/src/test/results/clientpositive/llap/tez_union_group_by.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/tez_union_group_by.q.out b/ql/src/test/results/clientpositive/llap/tez_union_group_by.q.out
index 36e7d92..5cb3652 100644
--- a/ql/src/test/results/clientpositive/llap/tez_union_group_by.q.out
+++ b/ql/src/test/results/clientpositive/llap/tez_union_group_by.q.out
@@ -145,6 +145,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Union 2 (CONTAINS)
         Map 5 <- Union 2 (CONTAINS)

http://git-wip-us.apache.org/repos/asf/hive/blob/cbfd2570/ql/src/test/results/clientpositive/llap/tez_union_multiinsert.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/tez_union_multiinsert.q.out b/ql/src/test/results/clientpositive/llap/tez_union_multiinsert.q.out
index 784a0c2..9145a63 100644
--- a/ql/src/test/results/clientpositive/llap/tez_union_multiinsert.q.out
+++ b/ql/src/test/results/clientpositive/llap/tez_union_multiinsert.q.out
@@ -55,6 +55,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-2
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 6 <- Union 3 (CONTAINS)
         Map 7 <- Union 3 (CONTAINS)
@@ -958,6 +959,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-2
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Union 2 (CONTAINS)
         Map 7 <- Union 2 (CONTAINS)
@@ -1857,6 +1859,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-2
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 1 <- Union 2 (CONTAINS)
         Map 7 <- Union 2 (CONTAINS)
@@ -2740,6 +2743,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-2
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 6 <- Union 3 (CONTAINS)
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Union 3 (CONTAINS)
@@ -3584,6 +3588,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-2
     Tez
+#### A masked pattern was here ####
       Edges:
         Map 6 <- Union 3 (CONTAINS)
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Union 3 (CONTAINS)

http://git-wip-us.apache.org/repos/asf/hive/blob/cbfd2570/ql/src/test/results/clientpositive/llap/tez_vector_dynpart_hashjoin_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/tez_vector_dynpart_hashjoin_1.q.out b/ql/src/test/results/clientpositive/llap/tez_vector_dynpart_hashjoin_1.q.out
index 6a57f1f..3264ff8 100644
--- a/ql/src/test/results/clientpositive/llap/tez_vector_dynpart_hashjoin_1.q.out
+++ b/ql/src/test/results/clientpositive/llap/tez_vector_dynpart_hashjoin_1.q.out
@@ -23,10 +23,10 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
-#### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
@@ -152,6 +152,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
@@ -276,6 +277,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
@@ -427,6 +429,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Map 4 (CUSTOM_SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
@@ -559,6 +562,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Map 4 (CUSTOM_SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
@@ -686,6 +690,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Map 5 (CUSTOM_SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)

http://git-wip-us.apache.org/repos/asf/hive/blob/cbfd2570/ql/src/test/results/clientpositive/llap/tez_vector_dynpart_hashjoin_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/tez_vector_dynpart_hashjoin_2.q.out b/ql/src/test/results/clientpositive/llap/tez_vector_dynpart_hashjoin_2.q.out
index 9eef871..0b0c8f6 100644
--- a/ql/src/test/results/clientpositive/llap/tez_vector_dynpart_hashjoin_2.q.out
+++ b/ql/src/test/results/clientpositive/llap/tez_vector_dynpart_hashjoin_2.q.out
@@ -33,10 +33,10 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
-#### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
@@ -224,6 +224,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
@@ -413,6 +414,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)

http://git-wip-us.apache.org/repos/asf/hive/blob/cbfd2570/ql/src/test/results/clientpositive/llap/vector_join_part_col_char.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vector_join_part_col_char.q.out b/ql/src/test/results/clientpositive/llap/vector_join_part_col_char.q.out
index 104c907..e81a158 100644
--- a/ql/src/test/results/clientpositive/llap/vector_join_part_col_char.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_join_part_col_char.q.out
@@ -108,6 +108,7 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
+#### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 3 (SIMPLE_EDGE)
 #### A masked pattern was here ####