You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by jc...@apache.org on 2020/08/02 14:39:29 UTC

[hive] branch master updated: HIVE-23939: SharedWorkOptimizer: take the union of columns in mergeable TableScans (Krisztian Kasa, reviewed by Jesus Camacho Rodriguez)

This is an automated email from the ASF dual-hosted git repository.

jcamacho 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 933c023  HIVE-23939: SharedWorkOptimizer: take the union of columns in mergeable TableScans (Krisztian Kasa, reviewed by Jesus Camacho Rodriguez)
933c023 is described below

commit 933c023b7d6524451bc6121eaa7b2f98827d23fa
Author: kasakrisz <kk...@cloudera.com>
AuthorDate: Sun Aug 2 16:39:11 2020 +0200

    HIVE-23939: SharedWorkOptimizer: take the union of columns in mergeable TableScans (Krisztian Kasa, reviewed by Jesus Camacho Rodriguez)
    
    Closes apache/hive#1324
---
 .../java/org/apache/hadoop/hive/conf/HiveConf.java |   4 +
 .../hive/ql/optimizer/SharedWorkOptimizer.java     | 584 +++++++------
 .../llap/annotate_stats_join_pkfk.q.out            |  12 +-
 .../llap/auto_join_reordering_values.q.out         | 146 +---
 .../llap/auto_join_without_localtask.q.out         |  80 +-
 .../clientpositive/llap/bucket_map_join_tez2.q.out | 176 ++--
 .../llap/cbo_SortUnionTransposeRule.q.out          |  16 +-
 .../llap/correlationoptimizer10.q.out              |  38 +-
 .../llap/correlationoptimizer3.q.out               | 152 +---
 .../llap/correlationoptimizer6.q.out               |  70 +-
 .../llap/correlationoptimizer7.q.out               |  60 +-
 .../clientpositive/llap/explainanalyze_2.q.out     |  55 +-
 .../clientpositive/llap/explainuser_1.q.out        |  85 +-
 .../clientpositive/llap/filter_cond_pushdown.q.out |  84 +-
 .../llap/filter_join_breaktask.q.out               |  92 +-
 .../clientpositive/llap/groupby_sort_1_23.q.out    |  61 +-
 .../llap/groupby_sort_skew_1_23.q.out              |  65 +-
 .../llap/hybridgrace_hashjoin_2.q.out              |  80 +-
 .../clientpositive/llap/infer_join_preds.q.out     |  13 +-
 .../test/results/clientpositive/llap/join12.q.out  |  33 +-
 .../test/results/clientpositive/llap/join2.q.out   |  40 +-
 .../test/results/clientpositive/llap/join3.q.out   |  35 +-
 .../clientpositive/llap/join_alt_syntax.q.out      |  24 +-
 .../results/clientpositive/llap/join_merging.q.out |  32 +-
 .../results/clientpositive/llap/join_parse.q.out   |  36 +-
 .../results/clientpositive/llap/keep_uniform.q.out | 290 +++----
 .../results/clientpositive/llap/leftsemijoin.q.out |  23 +-
 .../clientpositive/llap/limit_join_transpose.q.out |  20 +-
 .../results/clientpositive/llap/mapjoin_hint.q.out |  28 +-
 .../results/clientpositive/llap/masking_12.q.out   |  81 +-
 .../results/clientpositive/llap/masking_3.q.out    | 474 +++++------
 .../results/clientpositive/llap/masking_4.q.out    |  79 +-
 .../llap/nonblock_op_deduplicate.q.out             |  13 +-
 .../results/clientpositive/llap/ppd_join5.q.out    |  24 +-
 ql/src/test/results/clientpositive/llap/ptf.q.out  | 168 ++--
 .../clientpositive/llap/ptf_streaming.q.out        | 154 ++--
 .../llap/reduce_deduplicate_extended2.q.out        |  13 +-
 .../results/clientpositive/llap/semijoin.q.out     |  16 +-
 .../results/clientpositive/llap/semijoin2.q.out    | 132 ++-
 .../results/clientpositive/llap/semijoin4.q.out    |  60 +-
 .../results/clientpositive/llap/semijoin5.q.out    |  56 +-
 .../results/clientpositive/llap/sharedwork.q.out   | 104 +--
 .../clientpositive/llap/sharedworkresidual.q.out   |  37 +-
 .../sketches_rewrite_cume_dist_partition_by.q.out  |  13 +-
 .../llap/sketches_rewrite_ntile_partition_by.q.out |  13 +-
 .../llap/sketches_rewrite_rank_partition_by.q.out  |  13 +-
 .../llap/special_character_in_tabnames_1.q.out     | 248 ++----
 .../special_character_in_tabnames_quotes_1.q.out   | 248 ++----
 .../results/clientpositive/llap/subquery_ALL.q.out |  98 +--
 .../results/clientpositive/llap/subquery_ANY.q.out |  82 +-
 .../clientpositive/llap/subquery_exists.q.out      |  14 +-
 .../llap/subquery_exists_having.q.out              |  24 +-
 .../results/clientpositive/llap/subquery_in.q.out  | 669 +++++----------
 .../clientpositive/llap/subquery_in_having.q.out   |  86 +-
 .../clientpositive/llap/subquery_multi.q.out       | 619 +++++---------
 .../clientpositive/llap/subquery_multiinsert.q.out |  72 +-
 .../clientpositive/llap/subquery_notexists.q.out   | 119 ++-
 .../clientpositive/llap/subquery_notin.q.out       | 756 ++++++-----------
 .../clientpositive/llap/subquery_scalar.q.out      | 732 +++++-----------
 .../clientpositive/llap/subquery_select.q.out      | 941 ++++++++-------------
 .../clientpositive/llap/subquery_views.q.out       |  18 +-
 .../clientpositive/llap/tez_self_join.q.out        |  12 +-
 .../clientpositive/llap/unionall_lateralview.q.out |   2 +-
 .../clientpositive/llap/vector_join30.q.out        |  30 +-
 .../clientpositive/llap/vector_windowing.q.out     | 136 ++-
 .../clientpositive/llap/vectorized_ptf.q.out       | 270 +++---
 .../clientpositive/perf/tez/cbo_query23.q.out      |   4 +-
 .../perf/tez/constraints/cbo_query23.q.out         |   4 +-
 .../perf/tez/constraints/mv_query44.q.out          |  45 +-
 .../perf/tez/constraints/query11.q.out             | 144 ++--
 .../perf/tez/constraints/query14.q.out             | 800 +++++++++---------
 .../perf/tez/constraints/query18.q.out             |  29 +-
 .../perf/tez/constraints/query2.q.out              |  47 +-
 .../perf/tez/constraints/query23.q.out             | 306 ++++---
 .../perf/tez/constraints/query29.q.out             |  95 +--
 .../perf/tez/constraints/query33.q.out             |  95 +--
 .../perf/tez/constraints/query4.q.out              | 338 ++++----
 .../perf/tez/constraints/query41.q.out             |  23 +-
 .../perf/tez/constraints/query44.q.out             |  45 +-
 .../perf/tez/constraints/query45.q.out             |  67 +-
 .../perf/tez/constraints/query54.q.out             | 214 +++--
 .../perf/tez/constraints/query56.q.out             |  91 +-
 .../perf/tez/constraints/query58.q.out             |  96 +--
 .../perf/tez/constraints/query59.q.out             |  49 +-
 .../perf/tez/constraints/query60.q.out             |  91 +-
 .../perf/tez/constraints/query61.q.out             | 141 ++-
 .../perf/tez/constraints/query64.q.out             | 479 ++++++-----
 .../perf/tez/constraints/query70.q.out             |  65 +-
 .../perf/tez/constraints/query72.q.out             |  85 +-
 .../perf/tez/constraints/query74.q.out             | 103 ++-
 .../perf/tez/constraints/query8.q.out              |  47 +-
 .../perf/tez/constraints/query81.q.out             | 143 ++--
 .../perf/tez/constraints/query83.q.out             |  53 +-
 .../perf/tez/constraints/query9.q.out              | 228 +++--
 .../results/clientpositive/perf/tez/query14.q.out  | 760 ++++++++---------
 .../results/clientpositive/perf/tez/query18.q.out  |  69 +-
 .../results/clientpositive/perf/tez/query2.q.out   |  47 +-
 .../results/clientpositive/perf/tez/query23.q.out  | 338 ++++----
 .../results/clientpositive/perf/tez/query29.q.out  |  97 ++-
 .../results/clientpositive/perf/tez/query33.q.out  |  95 +--
 .../results/clientpositive/perf/tez/query41.q.out  |  23 +-
 .../results/clientpositive/perf/tez/query44.q.out  |  37 +-
 .../results/clientpositive/perf/tez/query45.q.out  |  85 +-
 .../results/clientpositive/perf/tez/query50.q.out  |  51 +-
 .../results/clientpositive/perf/tez/query54.q.out  | 236 +++---
 .../results/clientpositive/perf/tez/query56.q.out  |  95 +--
 .../results/clientpositive/perf/tez/query58.q.out  |  78 +-
 .../results/clientpositive/perf/tez/query59.q.out  | 106 ++-
 .../results/clientpositive/perf/tez/query60.q.out  |  95 +--
 .../results/clientpositive/perf/tez/query61.q.out  | 141 ++-
 .../results/clientpositive/perf/tez/query64.q.out  | 239 +++---
 .../results/clientpositive/perf/tez/query70.q.out  |  79 +-
 .../results/clientpositive/perf/tez/query72.q.out  | 124 ++-
 .../results/clientpositive/perf/tez/query8.q.out   |  47 +-
 .../results/clientpositive/perf/tez/query81.q.out  | 143 ++--
 .../results/clientpositive/perf/tez/query83.q.out  |  37 +-
 .../results/clientpositive/perf/tez/query9.q.out   | 228 +++--
 .../tez/hybridgrace_hashjoin_2.q.out               |  76 +-
 118 files changed, 6785 insertions(+), 9528 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 0e539d3..1fd5640 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -2551,6 +2551,10 @@ public class HiveConf extends Configuration {
         "Whether to enable shared work extended optimizer for semijoins. The optimizer tries to merge\n" +
         "scan operators if one of them reads the full table, even if the other one is the target for\n" +
         "one or more semijoin edges. Tez only."),
+    HIVE_SHARED_WORK_MERGE_TS_SCHEMA("hive.optimize.shared.work.merge.ts.schema", true,
+        "Whether to enable merging scan operators over the same table but with different schema." +
+            "The optimizer tries to merge the scan operators by taking the union of needed columns from " +
+            "all scan operators. Requires hive.optimize.shared.work to be set to true. Tez only."),
     HIVE_SHARED_WORK_REUSE_MAPJOIN_CACHE("hive.optimize.shared.work.mapjoin.cache.reuse", true,
         "When shared work optimizer is enabled, whether we should reuse the cache for the broadcast side\n" +
         "of mapjoin operators that share same broadcast input. Requires hive.optimize.shared.work\n" +
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SharedWorkOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SharedWorkOptimizer.java
index caed527..a19bf8a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SharedWorkOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SharedWorkOptimizer.java
@@ -160,7 +160,8 @@ public class SharedWorkOptimizer extends Transform {
     gatherDPPTableScanOps(pctx, optimizerCache);
 
     // Execute shared work optimization
-    sharedWorkOptimization(pctx, optimizerCache, tableNameToOps, sortedTables, false);
+    new SchemaAwareSharedWorkOptimizer().sharedWorkOptimization(
+        pctx, optimizerCache, tableNameToOps, sortedTables, false);
 
     if (LOG.isDebugEnabled()) {
       LOG.debug("After SharedWorkOptimizer:\n" + Operator.toString(pctx.getTopOps().values()));
@@ -183,7 +184,8 @@ public class SharedWorkOptimizer extends Transform {
       sortedTables = rankTablesByAccumulatedSize(pctx);
 
       // Execute shared work optimization with semijoin removal
-      boolean optimized = sharedWorkOptimization(pctx, optimizerCache, tableNameToOps, sortedTables, true);
+      boolean optimized = new SchemaAwareSharedWorkOptimizer().sharedWorkOptimization(
+          pctx, optimizerCache, tableNameToOps, sortedTables, true);
       if (optimized && pctx.getConf().getBoolVar(ConfVars.HIVE_SHARED_WORK_EXTENDED_OPTIMIZATION)) {
         // If it was further optimized, execute a second round of extended shared work optimizer
         sharedWorkExtendedOptimization(pctx, optimizerCache);
@@ -195,6 +197,15 @@ public class SharedWorkOptimizer extends Transform {
       }
     }
 
+    if (pctx.getConf().getBoolVar(ConfVars.HIVE_SHARED_WORK_MERGE_TS_SCHEMA)) {
+      new BaseSharedWorkOptimizer().sharedWorkOptimization(
+          pctx, optimizerCache, tableNameToOps, sortedTables, false);
+
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("After SharedWorkOptimizer merging TS schema:\n" + Operator.toString(pctx.getTopOps().values()));
+      }
+    }
+
     if(pctx.getConf().getBoolVar(ConfVars.HIVE_SHARED_WORK_REUSE_MAPJOIN_CACHE)) {
       // Try to reuse cache for broadcast side in mapjoin operators that
       // share same input.
@@ -273,258 +284,342 @@ public class SharedWorkOptimizer extends Transform {
     return pctx;
   }
 
-  private static boolean sharedWorkOptimization(ParseContext pctx, SharedWorkOptimizerCache optimizerCache,
-      ArrayListMultimap<String, TableScanOperator> tableNameToOps, List<Entry<String, Long>> sortedTables,
-      boolean removeSemijoin) throws SemanticException {
-    // Boolean to keep track of whether this method actually merged any TS operators
-    boolean mergedExecuted = false;
-
-    Multimap<String, TableScanOperator> existingOps = ArrayListMultimap.create();
-    Set<Operator<?>> removedOps = new HashSet<>();
-    for (Entry<String, Long> tablePair : sortedTables) {
-      String tableName = tablePair.getKey();
-      for (TableScanOperator discardableTsOp : tableNameToOps.get(tableName)) {
-        if (removedOps.contains(discardableTsOp)) {
-          LOG.debug("Skip {} as it has already been removed", discardableTsOp);
-          continue;
-        }
-        Collection<TableScanOperator> prevTsOps = existingOps.get(tableName);
-        for (TableScanOperator retainableTsOp : prevTsOps) {
-          if (removedOps.contains(retainableTsOp)) {
-            LOG.debug("Skip {} as it has already been removed", retainableTsOp);
+  /**
+   * Class wrapping shared work optimizer.
+   * This implementation enables merging of TS with different schemas by taking the union of the
+   * {@link TableScanDesc#getNeededColumns()} and {@link TableScanDesc#getNeededColumnIDs()}
+   * from both {@link TableScanOperator}s.
+   */
+  private static class BaseSharedWorkOptimizer {
+
+    public boolean sharedWorkOptimization(ParseContext pctx, SharedWorkOptimizerCache optimizerCache,
+                                           ArrayListMultimap<String, TableScanOperator> tableNameToOps, List<Entry<String, Long>> sortedTables,
+                                           boolean removeSemijoin) throws SemanticException {
+      // Boolean to keep track of whether this method actually merged any TS operators
+      boolean mergedExecuted = false;
+
+      Multimap<String, TableScanOperator> existingOps = ArrayListMultimap.create();
+      Set<Operator<?>> removedOps = new HashSet<>();
+      for (Entry<String, Long> tablePair : sortedTables) {
+        String tableName = tablePair.getKey();
+        for (TableScanOperator discardableTsOp : tableNameToOps.get(tableName)) {
+          if (removedOps.contains(discardableTsOp)) {
+            LOG.debug("Skip {} as it has already been removed", discardableTsOp);
             continue;
           }
-
-          SharedResult sr;
-          if (removeSemijoin) {
-            // We check if the two table scan operators can actually be merged modulo SJs.
-            // Hence, two conditions should be met:
-            // (i) the TS ops should be mergeable excluding any kind of DPP, and
-            // (ii) the DPP branches (excluding SJs) should be the same
-            boolean mergeable = areMergeable(pctx, optimizerCache, retainableTsOp, discardableTsOp);
-            if (!mergeable) {
-              // Skip
-              LOG.debug("{} and {} cannot be merged", retainableTsOp, discardableTsOp);
-              continue;
-            }
-            boolean validMerge = areMergeableExcludeSemijoinsExtendedCheck(
-                pctx, optimizerCache, retainableTsOp, discardableTsOp);
-            if (!validMerge) {
-              // Skip
-              LOG.debug("{} and {} do not meet preconditions", retainableTsOp, discardableTsOp);
+          Collection<TableScanOperator> prevTsOps = existingOps.get(tableName);
+          for (TableScanOperator retainableTsOp : prevTsOps) {
+            if (removedOps.contains(retainableTsOp)) {
+              LOG.debug("Skip {} as it has already been removed", retainableTsOp);
               continue;
             }
 
-            // If tests pass, we create the shared work optimizer additional information
-            // about the part of the tree that can be merged. We need to regenerate the
-            // cache because semijoin operators have been removed
-            sr = extractSharedOptimizationInfoForRoot(
-                pctx, optimizerCache, retainableTsOp, discardableTsOp);
-          } else {
-            // First we quickly check if the two table scan operators can actually be merged
-            if (!areMergeable(pctx, optimizerCache, retainableTsOp, discardableTsOp) ||
-                !areMergeableExtendedCheck(pctx, optimizerCache, retainableTsOp, discardableTsOp)) {
-              // Skip
-              LOG.debug("{} and {} cannot be merged", retainableTsOp, discardableTsOp);
-              continue;
-            }
+            SharedResult sr;
+            if (removeSemijoin) {
+              // We check if the two table scan operators can actually be merged modulo SJs.
+              // Hence, two conditions should be met:
+              // (i) the TS ops should be mergeable excluding any kind of DPP, and
+              // (ii) the DPP branches (excluding SJs) should be the same
+              boolean mergeable = areMergeable(pctx, retainableTsOp, discardableTsOp);
+              if (!mergeable) {
+                // Skip
+                LOG.debug("{} and {} cannot be merged", retainableTsOp, discardableTsOp);
+                continue;
+              }
+              boolean validMerge = areMergeableExcludeSemijoinsExtendedCheck(
+                  pctx, optimizerCache, retainableTsOp, discardableTsOp);
+              if (!validMerge) {
+                // Skip
+                LOG.debug("{} and {} do not meet preconditions", retainableTsOp, discardableTsOp);
+                continue;
+              }
 
-            // Secondly, we extract information about the part of the tree that can be merged
-            // as well as some structural information (memory consumption) that needs to be
-            // used to determined whether the merge can happen
-            sr = extractSharedOptimizationInfoForRoot(
-                pctx, optimizerCache, retainableTsOp, discardableTsOp);
+              // If tests pass, we create the shared work optimizer additional information
+              // about the part of the tree that can be merged. We need to regenerate the
+              // cache because semijoin operators have been removed
+              sr = extractSharedOptimizationInfoForRoot(
+                  pctx, optimizerCache, retainableTsOp, discardableTsOp);
+            } else {
+              // First we quickly check if the two table scan operators can actually be merged
+              if (!areMergeable(pctx, retainableTsOp, discardableTsOp) ||
+                  !areMergeableExtendedCheck(pctx, optimizerCache, retainableTsOp, discardableTsOp)) {
+                // Skip
+                LOG.debug("{} and {} cannot be merged", retainableTsOp, discardableTsOp);
+                continue;
+              }
 
-            // It seems these two operators can be merged.
-            // Check that plan meets some preconditions before doing it.
-            // In particular, in the presence of map joins in the upstream plan:
-            // - we cannot exceed the noconditional task size, and
-            // - if we already merged the big table, we cannot merge the broadcast
-            // tables.
-            if (!validPreConditions(pctx, optimizerCache, sr)) {
-              // Skip
-              LOG.debug("{} and {} do not meet preconditions", retainableTsOp, discardableTsOp);
-              continue;
+              // Secondly, we extract information about the part of the tree that can be merged
+              // as well as some structural information (memory consumption) that needs to be
+              // used to determined whether the merge can happen
+              sr = extractSharedOptimizationInfoForRoot(
+                  pctx, optimizerCache, retainableTsOp, discardableTsOp);
+
+              // It seems these two operators can be merged.
+              // Check that plan meets some preconditions before doing it.
+              // In particular, in the presence of map joins in the upstream plan:
+              // - we cannot exceed the noconditional task size, and
+              // - if we already merged the big table, we cannot merge the broadcast
+              // tables.
+              if (!validPreConditions(pctx, optimizerCache, sr)) {
+                // Skip
+                LOG.debug("{} and {} do not meet preconditions", retainableTsOp, discardableTsOp);
+                continue;
+              }
             }
-          }
 
-          // We can merge
-          mergedExecuted = true;
-          if (sr.retainableOps.size() > 1) {
-            // More than TS operator
-            Operator<?> lastRetainableOp = sr.retainableOps.get(sr.retainableOps.size() - 1);
-            Operator<?> lastDiscardableOp = sr.discardableOps.get(sr.discardableOps.size() - 1);
-            if (lastDiscardableOp.getNumChild() != 0) {
-              List<Operator<? extends OperatorDesc>> allChildren =
-                  Lists.newArrayList(lastDiscardableOp.getChildOperators());
-              for (Operator<? extends OperatorDesc> op : allChildren) {
-                lastDiscardableOp.getChildOperators().remove(op);
-                op.replaceParent(lastDiscardableOp, lastRetainableOp);
-                lastRetainableOp.getChildOperators().add(op);
+            // We can merge
+            mergedExecuted = true;
+            if (sr.retainableOps.size() > 1) {
+              // More than TS operator
+              Operator<?> lastRetainableOp = sr.retainableOps.get(sr.retainableOps.size() - 1);
+              Operator<?> lastDiscardableOp = sr.discardableOps.get(sr.discardableOps.size() - 1);
+              if (lastDiscardableOp.getNumChild() != 0) {
+                List<Operator<? extends OperatorDesc>> allChildren =
+                    Lists.newArrayList(lastDiscardableOp.getChildOperators());
+                for (Operator<? extends OperatorDesc> op : allChildren) {
+                  lastDiscardableOp.getChildOperators().remove(op);
+                  op.replaceParent(lastDiscardableOp, lastRetainableOp);
+                  lastRetainableOp.getChildOperators().add(op);
+                }
               }
-            }
 
-            LOG.debug("Merging subtree starting at {} into subtree starting at {}",
-                discardableTsOp, retainableTsOp);
-          } else {
-            ExprNodeDesc newRetainableTsFilterExpr = null;
-            List<ExprNodeDesc> semijoinExprNodes = new ArrayList<>();
-            if (retainableTsOp.getConf().getFilterExpr() != null) {
-              // Gather SJ expressions and normal expressions
-              List<ExprNodeDesc> allExprNodesExceptSemijoin = new ArrayList<>();
-              splitExpressions(retainableTsOp.getConf().getFilterExpr(),
-                  allExprNodesExceptSemijoin, semijoinExprNodes);
-              // Create new expressions
-              if (allExprNodesExceptSemijoin.size() > 1) {
-                newRetainableTsFilterExpr = ExprNodeGenericFuncDesc.newInstance(
-                    new GenericUDFOPAnd(), allExprNodesExceptSemijoin);
-              } else if (allExprNodesExceptSemijoin.size() > 0 &&
-                  allExprNodesExceptSemijoin.get(0) instanceof ExprNodeGenericFuncDesc) {
-                newRetainableTsFilterExpr = allExprNodesExceptSemijoin.get(0);
+              LOG.debug("Merging subtree starting at {} into subtree starting at {}",
+                  discardableTsOp, retainableTsOp);
+            } else {
+              ExprNodeDesc newRetainableTsFilterExpr = null;
+              List<ExprNodeDesc> semijoinExprNodes = new ArrayList<>();
+              if (retainableTsOp.getConf().getFilterExpr() != null) {
+                // Gather SJ expressions and normal expressions
+                List<ExprNodeDesc> allExprNodesExceptSemijoin = new ArrayList<>();
+                splitExpressions(retainableTsOp.getConf().getFilterExpr(),
+                    allExprNodesExceptSemijoin, semijoinExprNodes);
+                // Create new expressions
+                if (allExprNodesExceptSemijoin.size() > 1) {
+                  newRetainableTsFilterExpr = ExprNodeGenericFuncDesc.newInstance(
+                      new GenericUDFOPAnd(), allExprNodesExceptSemijoin);
+                } else if (allExprNodesExceptSemijoin.size() > 0 &&
+                    allExprNodesExceptSemijoin.get(0) instanceof ExprNodeGenericFuncDesc) {
+                  newRetainableTsFilterExpr = allExprNodesExceptSemijoin.get(0);
+                }
+                // Push filter on top of children for retainable
+                pushFilterToTopOfTableScan(optimizerCache, retainableTsOp);
               }
-              // Push filter on top of children for retainable
-              pushFilterToTopOfTableScan(optimizerCache, retainableTsOp);
-            }
-            ExprNodeDesc newDiscardableTsFilterExpr = null;
-            if (discardableTsOp.getConf().getFilterExpr() != null) {
-              // If there is a single discardable operator, it is a TableScanOperator
-              // and it means that we will merge filter expressions for it. Thus, we
-              // might need to remove DPP predicates before doing that
-              List<ExprNodeDesc> allExprNodesExceptSemijoin = new ArrayList<>();
-              splitExpressions(discardableTsOp.getConf().getFilterExpr(),
-                  allExprNodesExceptSemijoin, new ArrayList<>());
-              // Create new expressions
-              if (allExprNodesExceptSemijoin.size() > 1) {
-                newDiscardableTsFilterExpr = ExprNodeGenericFuncDesc.newInstance(
-                    new GenericUDFOPAnd(), allExprNodesExceptSemijoin);
-              } else if (allExprNodesExceptSemijoin.size() > 0 &&
-                  allExprNodesExceptSemijoin.get(0) instanceof ExprNodeGenericFuncDesc) {
-                newDiscardableTsFilterExpr = allExprNodesExceptSemijoin.get(0);
+              ExprNodeDesc newDiscardableTsFilterExpr = null;
+              if (discardableTsOp.getConf().getFilterExpr() != null) {
+                // If there is a single discardable operator, it is a TableScanOperator
+                // and it means that we will merge filter expressions for it. Thus, we
+                // might need to remove DPP predicates before doing that
+                List<ExprNodeDesc> allExprNodesExceptSemijoin = new ArrayList<>();
+                splitExpressions(discardableTsOp.getConf().getFilterExpr(),
+                    allExprNodesExceptSemijoin, new ArrayList<>());
+                // Create new expressions
+                if (allExprNodesExceptSemijoin.size() > 1) {
+                  newDiscardableTsFilterExpr = ExprNodeGenericFuncDesc.newInstance(
+                      new GenericUDFOPAnd(), allExprNodesExceptSemijoin);
+                } else if (allExprNodesExceptSemijoin.size() > 0 &&
+                    allExprNodesExceptSemijoin.get(0) instanceof ExprNodeGenericFuncDesc) {
+                  newDiscardableTsFilterExpr = allExprNodesExceptSemijoin.get(0);
+                }
+                // Remove and add semijoin filter from expressions
+                replaceSemijoinExpressions(discardableTsOp, semijoinExprNodes);
+                // Push filter on top of children for discardable
+                pushFilterToTopOfTableScan(optimizerCache, discardableTsOp);
               }
-              // Remove and add semijoin filter from expressions
-              replaceSemijoinExpressions(discardableTsOp, semijoinExprNodes);
-              // Push filter on top of children for discardable
-              pushFilterToTopOfTableScan(optimizerCache, discardableTsOp);
-            }
-            // Obtain filter for shared TS operator
-            ExprNodeGenericFuncDesc exprNode = null;
-            if (newRetainableTsFilterExpr != null && newDiscardableTsFilterExpr != null) {
-              // Combine
-              exprNode = (ExprNodeGenericFuncDesc) newRetainableTsFilterExpr;
-              if (!exprNode.isSame(newDiscardableTsFilterExpr)) {
-                // We merge filters from previous scan by ORing with filters from current scan
-                if (exprNode.getGenericUDF() instanceof GenericUDFOPOr) {
-                  List<ExprNodeDesc> newChildren = new ArrayList<>(exprNode.getChildren().size() + 1);
-                  for (ExprNodeDesc childExprNode : exprNode.getChildren()) {
-                    if (childExprNode.isSame(newDiscardableTsFilterExpr)) {
-                      // We do not need to do anything, it is in the OR expression
-                      break;
+              // Obtain filter for shared TS operator
+              ExprNodeGenericFuncDesc exprNode = null;
+              if (newRetainableTsFilterExpr != null && newDiscardableTsFilterExpr != null) {
+                // Combine
+                exprNode = (ExprNodeGenericFuncDesc) newRetainableTsFilterExpr;
+                if (!exprNode.isSame(newDiscardableTsFilterExpr)) {
+                  // We merge filters from previous scan by ORing with filters from current scan
+                  if (exprNode.getGenericUDF() instanceof GenericUDFOPOr) {
+                    List<ExprNodeDesc> newChildren = new ArrayList<>(exprNode.getChildren().size() + 1);
+                    for (ExprNodeDesc childExprNode : exprNode.getChildren()) {
+                      if (childExprNode.isSame(newDiscardableTsFilterExpr)) {
+                        // We do not need to do anything, it is in the OR expression
+                        break;
+                      }
+                      newChildren.add(childExprNode);
                     }
-                    newChildren.add(childExprNode);
-                  }
-                  if (exprNode.getChildren().size() == newChildren.size()) {
-                    newChildren.add(newDiscardableTsFilterExpr);
+                    if (exprNode.getChildren().size() == newChildren.size()) {
+                      newChildren.add(newDiscardableTsFilterExpr);
+                      exprNode = ExprNodeGenericFuncDesc.newInstance(
+                          new GenericUDFOPOr(),
+                          newChildren);
+                    }
+                  } else {
                     exprNode = ExprNodeGenericFuncDesc.newInstance(
                         new GenericUDFOPOr(),
-                        newChildren);
+                        Arrays.asList(exprNode, newDiscardableTsFilterExpr));
                   }
-                } else {
-                  exprNode = ExprNodeGenericFuncDesc.newInstance(
-                      new GenericUDFOPOr(),
-                      Arrays.asList(exprNode, newDiscardableTsFilterExpr));
                 }
               }
-            }
-            // Create expression node that will be used for the retainable table scan
-            if (!semijoinExprNodes.isEmpty()) {
-              if (exprNode != null) {
-                semijoinExprNodes.add(0, exprNode);
+              // Create expression node that will be used for the retainable table scan
+              if (!semijoinExprNodes.isEmpty()) {
+                if (exprNode != null) {
+                  semijoinExprNodes.add(0, exprNode);
+                }
+                if (semijoinExprNodes.size() > 1) {
+                  exprNode = ExprNodeGenericFuncDesc.newInstance(
+                      new GenericUDFOPAnd(), semijoinExprNodes);
+                } else {
+                  exprNode = (ExprNodeGenericFuncDesc) semijoinExprNodes.get(0);
+                }
               }
-              if (semijoinExprNodes.size() > 1) {
-                exprNode = ExprNodeGenericFuncDesc.newInstance(
-                    new GenericUDFOPAnd(), semijoinExprNodes);
-              } else {
-                exprNode = (ExprNodeGenericFuncDesc) semijoinExprNodes.get(0);
+              // Replace filter
+              retainableTsOp.getConf().setFilterExpr(exprNode);
+              // Replace table scan operator
+              List<Operator<? extends OperatorDesc>> allChildren =
+                  Lists.newArrayList(discardableTsOp.getChildOperators());
+              for (Operator<? extends OperatorDesc> op : allChildren) {
+                discardableTsOp.getChildOperators().remove(op);
+                op.replaceParent(discardableTsOp, retainableTsOp);
+                retainableTsOp.getChildOperators().add(op);
               }
+
+              LOG.debug("Merging {} into {}", discardableTsOp, retainableTsOp);
             }
-            // Replace filter
-            retainableTsOp.getConf().setFilterExpr(exprNode);
-            // Replace table scan operator
-            List<Operator<? extends OperatorDesc>> allChildren =
-                Lists.newArrayList(discardableTsOp.getChildOperators());
-            for (Operator<? extends OperatorDesc> op : allChildren) {
-              discardableTsOp.getChildOperators().remove(op);
-              op.replaceParent(discardableTsOp, retainableTsOp);
-              retainableTsOp.getChildOperators().add(op);
+
+            // First we remove the input operators of the expression that
+            // we are going to eliminate
+            for (Operator<?> op : sr.discardableInputOps) {
+              OperatorUtils.removeOperator(op);
+              optimizerCache.removeOp(op);
+              removedOps.add(op);
+              // Remove DPP predicates
+              if (op instanceof ReduceSinkOperator) {
+                SemiJoinBranchInfo sjbi = pctx.getRsToSemiJoinBranchInfo().get(op);
+                if (sjbi != null && !sr.discardableOps.contains(sjbi.getTsOp()) &&
+                    !sr.discardableInputOps.contains(sjbi.getTsOp())) {
+                  GenTezUtils.removeSemiJoinOperator(
+                      pctx, (ReduceSinkOperator) op, sjbi.getTsOp());
+                  optimizerCache.tableScanToDPPSource.remove(sjbi.getTsOp(), op);
+                }
+              } else if (op instanceof AppMasterEventOperator) {
+                DynamicPruningEventDesc dped = (DynamicPruningEventDesc) op.getConf();
+                if (!sr.discardableOps.contains(dped.getTableScan()) &&
+                    !sr.discardableInputOps.contains(dped.getTableScan())) {
+                  GenTezUtils.removeSemiJoinOperator(
+                      pctx, (AppMasterEventOperator) op, dped.getTableScan());
+                  optimizerCache.tableScanToDPPSource.remove(dped.getTableScan(), op);
+                }
+              }
+              LOG.debug("Input operator removed: {}", op);
             }
 
-            LOG.debug("Merging {} into {}", discardableTsOp, retainableTsOp);
-          }
+            // A shared TSop across branches can not have probeContext that utilizes single branch info
+            // Filtered-out rows from one branch might be needed by another branch sharing a TSop
+            if (retainableTsOp.getProbeDecodeContext() != null) {
+              LOG.debug("Removing probeDecodeCntx for merged TS op {}", retainableTsOp);
+              retainableTsOp.setProbeDecodeContext(null);
+              retainableTsOp.getConf().setProbeDecodeContext(null);
+            }
 
-          // First we remove the input operators of the expression that
-          // we are going to eliminate
-          for (Operator<?> op : sr.discardableInputOps) {
-            OperatorUtils.removeOperator(op);
-            optimizerCache.removeOp(op);
-            removedOps.add(op);
-            // Remove DPP predicates
-            if (op instanceof ReduceSinkOperator) {
-              SemiJoinBranchInfo sjbi = pctx.getRsToSemiJoinBranchInfo().get(op);
-              if (sjbi != null && !sr.discardableOps.contains(sjbi.getTsOp()) &&
-                  !sr.discardableInputOps.contains(sjbi.getTsOp())) {
-                GenTezUtils.removeSemiJoinOperator(
-                    pctx, (ReduceSinkOperator) op, sjbi.getTsOp());
-                optimizerCache.tableScanToDPPSource.remove(sjbi.getTsOp(), op);
-              }
-            } else if (op instanceof AppMasterEventOperator) {
-              DynamicPruningEventDesc dped = (DynamicPruningEventDesc) op.getConf();
-              if (!sr.discardableOps.contains(dped.getTableScan()) &&
-                  !sr.discardableInputOps.contains(dped.getTableScan())) {
-                GenTezUtils.removeSemiJoinOperator(
-                    pctx, (AppMasterEventOperator) op, dped.getTableScan());
-                optimizerCache.tableScanToDPPSource.remove(dped.getTableScan(), op);
-              }
+            // Then we merge the operators of the works we are going to merge
+            mergeSchema(discardableTsOp, retainableTsOp);
+            optimizerCache.removeOpAndCombineWork(discardableTsOp, retainableTsOp);
+            removedOps.add(discardableTsOp);
+            // Finally we remove the expression from the tree
+            for (Operator<?> op : sr.discardableOps) {
+              OperatorUtils.removeOperator(op);
+              optimizerCache.removeOp(op);
+              removedOps.add(op);
+              LOG.debug("Operator removed: {}", op);
             }
-            LOG.debug("Input operator removed: {}", op);
-          }
 
-          // A shared TSop across branches can not have probeContext that utilizes single branch info
-          // Filtered-out rows from one branch might be needed by another branch sharing a TSop
-          if (retainableTsOp.getProbeDecodeContext() != null) {
-            LOG.debug("Removing probeDecodeCntx for merged TS op {}", retainableTsOp);
-            retainableTsOp.setProbeDecodeContext(null);
-            retainableTsOp.getConf().setProbeDecodeContext(null);
+            break;
           }
 
-          // Then we merge the operators of the works we are going to merge
-          optimizerCache.removeOpAndCombineWork(discardableTsOp, retainableTsOp);
-          removedOps.add(discardableTsOp);
-          // Finally we remove the expression from the tree
-          for (Operator<?> op : sr.discardableOps) {
-            OperatorUtils.removeOperator(op);
-            optimizerCache.removeOp(op);
-            removedOps.add(op);
-            LOG.debug("Operator removed: {}", op);
+          if (removedOps.contains(discardableTsOp)) {
+            // This operator has been removed, remove it from the list of existing operators
+            existingOps.remove(tableName, discardableTsOp);
+          } else {
+            // This operator has not been removed, include it in the list of existing operators
+            existingOps.put(tableName, discardableTsOp);
           }
-
-          break;
         }
+      }
 
-        if (removedOps.contains(discardableTsOp)) {
-          // This operator has been removed, remove it from the list of existing operators
-          existingOps.remove(tableName, discardableTsOp);
-        } else {
-          // This operator has not been removed, include it in the list of existing operators
-          existingOps.put(tableName, discardableTsOp);
+      // Remove unused table scan operators
+      pctx.getTopOps().entrySet().removeIf(
+          (Entry<String, TableScanOperator> e) -> e.getValue().getNumChild() == 0);
+
+      return mergedExecuted;
+    }
+
+    // FIXME: probably this should also be integrated with isSame() logics
+    protected boolean areMergeable(ParseContext pctx, TableScanOperator tsOp1, TableScanOperator tsOp2)
+        throws SemanticException {
+      // If row limit does not match, we currently do not merge
+      if (tsOp1.getConf().getRowLimit() != tsOp2.getConf().getRowLimit()) {
+        return false;
+      }
+      // If table properties do not match, we currently do not merge
+      if (!Objects.equals(tsOp1.getConf().getOpProps(), tsOp2.getConf().getOpProps())) {
+        return false;
+      }
+      // If partitions do not match, we currently do not merge
+      PrunedPartitionList prevTsOpPPList = pctx.getPrunedPartitions(tsOp1);
+      PrunedPartitionList tsOpPPList = pctx.getPrunedPartitions(tsOp2);
+      if (!prevTsOpPPList.getPartitions().equals(tsOpPPList.getPartitions())) {
+        return false;
+      }
+
+      if(!Objects.equals(tsOp1.getConf().getIncludedBuckets(),
+          tsOp2.getConf().getIncludedBuckets())) {
+        return false;
+      }
+
+      return true;
+    }
+
+    protected void mergeSchema(TableScanOperator discardableTsOp, TableScanOperator retainableTsOp) {
+      for (int colId : discardableTsOp.getConf().getNeededColumnIDs()) {
+        if (!retainableTsOp.getConf().getNeededColumnIDs().contains(colId)) {
+          retainableTsOp.getConf().getNeededColumnIDs().add(colId);
+        }
+      }
+      for (String col : discardableTsOp.getConf().getNeededColumns()) {
+        if (!retainableTsOp.getConf().getNeededColumns().contains(col)) {
+          retainableTsOp.getConf().getNeededColumns().add(col);
         }
       }
     }
+  }
 
-    // Remove unused table scan operators
-    pctx.getTopOps().entrySet().removeIf(
-        (Entry<String, TableScanOperator> e) -> e.getValue().getNumChild() == 0);
+  /**
+   * More strict implementation of shared work optimizer.
+   * This implementation doesn't merge {@link TableScanOperator}s with different schema.
+   */
+  private static class SchemaAwareSharedWorkOptimizer extends BaseSharedWorkOptimizer {
+    @Override
+    protected boolean areMergeable(ParseContext pctx, TableScanOperator tsOp1, TableScanOperator tsOp2)
+        throws SemanticException {
+      // First we check if the two table scan operators can actually be merged
+      // If schemas do not match, we currently do not merge
+      List<String> prevTsOpNeededColumns = tsOp1.getNeededColumns();
+      List<String> tsOpNeededColumns = tsOp2.getNeededColumns();
+      if (prevTsOpNeededColumns.size() != tsOpNeededColumns.size()) {
+        return false;
+      }
+      boolean notEqual = false;
+      for (int i = 0; i < prevTsOpNeededColumns.size(); i++) {
+        if (!prevTsOpNeededColumns.get(i).equals(tsOpNeededColumns.get(i))) {
+          notEqual = true;
+          break;
+        }
+      }
+      if (notEqual) {
+        return false;
+      }
+
+      return super.areMergeable(pctx, tsOp1, tsOp2);
+    }
 
-    return mergedExecuted;
+    @Override
+    protected void mergeSchema(TableScanOperator discardableTsOp, TableScanOperator retainableTsOp) {
+      // nop
+    }
   }
 
   /**
@@ -899,49 +994,6 @@ public class SharedWorkOptimizer extends Transform {
     return sortedOps;
   }
 
-  // FIXME: probably this should also be integrated with isSame() logics
-  private static boolean areMergeable(ParseContext pctx, SharedWorkOptimizerCache optimizerCache,
-          TableScanOperator tsOp1, TableScanOperator tsOp2) throws SemanticException {
-    // First we check if the two table scan operators can actually be merged
-    // If schemas do not match, we currently do not merge
-    List<String> prevTsOpNeededColumns = tsOp1.getNeededColumns();
-    List<String> tsOpNeededColumns = tsOp2.getNeededColumns();
-    if (prevTsOpNeededColumns.size() != tsOpNeededColumns.size()) {
-      return false;
-    }
-    boolean notEqual = false;
-    for (int i = 0; i < prevTsOpNeededColumns.size(); i++) {
-      if (!prevTsOpNeededColumns.get(i).equals(tsOpNeededColumns.get(i))) {
-        notEqual = true;
-        break;
-      }
-    }
-    if (notEqual) {
-      return false;
-    }
-    // If row limit does not match, we currently do not merge
-    if (tsOp1.getConf().getRowLimit() != tsOp2.getConf().getRowLimit()) {
-      return false;
-    }
-    // If table properties do not match, we currently do not merge
-    if (!Objects.equals(tsOp1.getConf().getOpProps(), tsOp2.getConf().getOpProps())) {
-      return false;
-    }
-    // If partitions do not match, we currently do not merge
-    PrunedPartitionList prevTsOpPPList = pctx.getPrunedPartitions(tsOp1);
-    PrunedPartitionList tsOpPPList = pctx.getPrunedPartitions(tsOp2);
-    if (!prevTsOpPPList.getPartitions().equals(tsOpPPList.getPartitions())) {
-      return false;
-    }
-
-    if(!Objects.equals(tsOp1.getConf().getIncludedBuckets(),
-        tsOp2.getConf().getIncludedBuckets())) {
-      return false;
-    }
-
-    return true;
-  }
-
   private static boolean areMergeableExtendedCheck(ParseContext pctx, SharedWorkOptimizerCache optimizerCache,
       TableScanOperator tsOp1, TableScanOperator tsOp2) throws SemanticException {
     // If is a DPP, check if actually it refers to same target, column, etc.
@@ -1140,6 +1192,11 @@ public class SharedWorkOptimizer extends Transform {
       return new SharedResult(retainableOps, discardableOps, discardableInputOps,
           dataSize, maxDataSize);
     }
+    if (retainableTsOp.getChildOperators().size() == 0 || discardableTsOp.getChildOperators().size() == 0) {
+      return new SharedResult(retainableOps, discardableOps, discardableInputOps,
+          dataSize, maxDataSize);
+    }
+
     Operator<?> currentOp1 = retainableTsOp.getChildOperators().get(0);
     Operator<?> currentOp2 = discardableTsOp.getChildOperators().get(0);
 
@@ -1789,11 +1846,16 @@ public class SharedWorkOptimizer extends Transform {
               .stream()
               .map(ExprNodeDesc::getExprString)
               .collect(Collectors.toSet());
+          ExprNodeGenericFuncDesc genericFuncDesc = (ExprNodeGenericFuncDesc) filterExprNode;
+          List<ExprNodeDesc> newChildren = new ArrayList<>(
+              filterExprNode.getChildren().size() + tableScanExprNode.getChildren().size());
+          newChildren.addAll(filterExprNode.getChildren());
           for (ExprNodeDesc e : tableScanExprNode.getChildren()) {
             if (visitedExprs.add(e.getExprString())) {
-              filterExprNode.getChildren().add(e.clone());
+              newChildren.add(e.clone());
             }
           }
+          genericFuncDesc.setChildren(newChildren);
         } else if (isOpAndFilter) {
           Set<String> visitedExprs = filterExprNode.getChildren()
               .stream()
diff --git a/ql/src/test/results/clientpositive/llap/annotate_stats_join_pkfk.q.out b/ql/src/test/results/clientpositive/llap/annotate_stats_join_pkfk.q.out
index 0b1d492..0737da7 100644
--- a/ql/src/test/results/clientpositive/llap/annotate_stats_join_pkfk.q.out
+++ b/ql/src/test/results/clientpositive/llap/annotate_stats_join_pkfk.q.out
@@ -1148,7 +1148,7 @@ STAGE PLANS:
 #### 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)
+        Reducer 3 <- Map 4 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -1176,7 +1176,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: s
-                  filterExpr: ((s_floor_space > 1000) and s_store_sk is not null) (type: boolean)
+                  filterExpr: (((s_floor_space > 1000) and s_store_sk is not null) or s_store_sk is not null) (type: boolean)
                   Statistics: Num rows: 12 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((s_floor_space > 1000) and s_store_sk is not null) (type: boolean)
@@ -1191,14 +1191,6 @@ STAGE PLANS:
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 12 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 5 
-            Map Operator Tree:
-                TableScan
-                  alias: s1
-                  filterExpr: s_store_sk is not null (type: boolean)
-                  Statistics: Num rows: 12 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: s_store_sk is not null (type: boolean)
                     Statistics: Num rows: 12 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/auto_join_reordering_values.q.out b/ql/src/test/results/clientpositive/llap/auto_join_reordering_values.q.out
index 18f394b..15ad6c6 100644
--- a/ql/src/test/results/clientpositive/llap/auto_join_reordering_values.q.out
+++ b/ql/src/test/results/clientpositive/llap/auto_join_reordering_values.q.out
@@ -113,16 +113,16 @@ STAGE PLANS:
 #### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 6 (SIMPLE_EDGE)
-        Reducer 3 <- Map 7 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
-        Reducer 4 <- Map 8 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
-        Reducer 5 <- Map 9 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+        Reducer 4 <- Map 1 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+        Reducer 5 <- Map 7 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: orderpayment
-                  filterExpr: (date is not null and dealid is not null and cityid is not null and userid is not null) (type: boolean)
+                  filterExpr: ((date is not null and dealid is not null and cityid is not null and userid is not null) or dealid is not null or cityid is not null) (type: boolean)
                   Statistics: Num rows: 1 Data size: 106 Basic stats: COMPLETE Column stats: COMPLETE
                   GatherStats: false
                   Filter Operator
@@ -144,122 +144,30 @@ STAGE PLANS:
                         tag: 0
                         value expressions: _col0 (type: int), _col2 (type: int), _col3 (type: int)
                         auto parallelism: true
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-            Path -> Alias:
-#### A masked pattern was here ####
-            Path -> Partition:
-#### A masked pattern was here ####
-                Partition
-                  base file name: orderpayment_small
-                  input format: org.apache.hadoop.mapred.TextInputFormat
-                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                  properties:
-                    bucket_count -1
-                    bucketing_version 2
-                    column.name.delimiter ,
-                    columns dealid,date,time,cityid,userid
-                    columns.types int:string:string:int:int
-#### A masked pattern was here ####
-                    name default.orderpayment_small
-                    serialization.format 1
-                    serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                
-                    input format: org.apache.hadoop.mapred.TextInputFormat
-                    output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                    properties:
-                      bucketing_version 2
-                      column.name.delimiter ,
-                      columns dealid,date,time,cityid,userid
-                      columns.comments 
-                      columns.types int:string:string:int:int
-#### A masked pattern was here ####
-                      name default.orderpayment_small
-                      serialization.format 1
-                      serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                    name: default.orderpayment_small
-                  name: default.orderpayment_small
-            Truncated Path -> Alias:
-              /orderpayment_small [orderpayment]
-        Map 6 
-            Map Operator Tree:
-                TableScan
-                  alias: dim_pay_date
-                  filterExpr: date is not null (type: boolean)
-                  Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: COMPLETE
-                  GatherStats: false
                   Filter Operator
                     isSamplingPred: false
-                    predicate: date is not null (type: boolean)
-                    Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: COMPLETE
+                    predicate: dealid is not null (type: boolean)
+                    Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
-                      expressions: date (type: string)
+                      expressions: dealid (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         bucketingVersion: 2
-                        key expressions: _col0 (type: string)
+                        key expressions: _col0 (type: int)
                         null sort order: z
                         numBuckets: -1
                         sort order: +
-                        Map-reduce partition columns: _col0 (type: string)
-                        Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: COMPLETE
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
                         tag: 1
                         auto parallelism: true
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-            Path -> Alias:
-#### A masked pattern was here ####
-            Path -> Partition:
-#### A masked pattern was here ####
-                Partition
-                  base file name: orderpayment_small
-                  input format: org.apache.hadoop.mapred.TextInputFormat
-                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                  properties:
-                    bucket_count -1
-                    bucketing_version 2
-                    column.name.delimiter ,
-                    columns dealid,date,time,cityid,userid
-                    columns.types int:string:string:int:int
-#### A masked pattern was here ####
-                    name default.orderpayment_small
-                    serialization.format 1
-                    serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                
-                    input format: org.apache.hadoop.mapred.TextInputFormat
-                    output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                    properties:
-                      bucketing_version 2
-                      column.name.delimiter ,
-                      columns dealid,date,time,cityid,userid
-                      columns.comments 
-                      columns.types int:string:string:int:int
-#### A masked pattern was here ####
-                      name default.orderpayment_small
-                      serialization.format 1
-                      serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                    name: default.orderpayment_small
-                  name: default.orderpayment_small
-            Truncated Path -> Alias:
-              /orderpayment_small [dim_pay_date]
-        Map 7 
-            Map Operator Tree:
-                TableScan
-                  alias: deal
-                  filterExpr: dealid is not null (type: boolean)
-                  Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
-                  GatherStats: false
                   Filter Operator
                     isSamplingPred: false
-                    predicate: dealid is not null (type: boolean)
+                    predicate: cityid is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
-                      expressions: dealid (type: int)
+                      expressions: cityid (type: int)
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
@@ -310,30 +218,30 @@ STAGE PLANS:
                     name: default.orderpayment_small
                   name: default.orderpayment_small
             Truncated Path -> Alias:
-              /orderpayment_small [deal]
-        Map 8 
+              /orderpayment_small [orderpayment]
+        Map 6 
             Map Operator Tree:
                 TableScan
-                  alias: order_city
-                  filterExpr: cityid is not null (type: boolean)
-                  Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                  alias: dim_pay_date
+                  filterExpr: date is not null (type: boolean)
+                  Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: COMPLETE
                   GatherStats: false
                   Filter Operator
                     isSamplingPred: false
-                    predicate: cityid is not null (type: boolean)
-                    Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                    predicate: date is not null (type: boolean)
+                    Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
-                      expressions: cityid (type: int)
+                      expressions: date (type: string)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         bucketingVersion: 2
-                        key expressions: _col0 (type: int)
+                        key expressions: _col0 (type: string)
                         null sort order: z
                         numBuckets: -1
                         sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: COMPLETE
                         tag: 1
                         auto parallelism: true
             Execution mode: vectorized, llap
@@ -374,8 +282,8 @@ STAGE PLANS:
                     name: default.orderpayment_small
                   name: default.orderpayment_small
             Truncated Path -> Alias:
-              /orderpayment_small [order_city]
-        Map 9 
+              /orderpayment_small [dim_pay_date]
+        Map 7 
             Map Operator Tree:
                 TableScan
                   alias: user
diff --git a/ql/src/test/results/clientpositive/llap/auto_join_without_localtask.q.out b/ql/src/test/results/clientpositive/llap/auto_join_without_localtask.q.out
index 06613aa..b618440 100644
--- a/ql/src/test/results/clientpositive/llap/auto_join_without_localtask.q.out
+++ b/ql/src/test/results/clientpositive/llap/auto_join_without_localtask.q.out
@@ -179,7 +179,7 @@ STAGE PLANS:
 #### 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)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
         Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
@@ -187,7 +187,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: a
-                  filterExpr: (key is not null and value is not null) (type: boolean)
+                  filterExpr: ((key is not null and value is not null) or value is not null) (type: boolean)
                   Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key is not null and value is not null) (type: boolean)
@@ -203,6 +203,19 @@ STAGE PLANS:
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: string)
+                  Filter Operator
+                    predicate: value is not null (type: boolean)
+                    Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: value (type: string)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: llap
             LLAP IO: no inputs
         Map 5 
@@ -226,27 +239,6 @@ STAGE PLANS:
                         Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: llap
             LLAP IO: no inputs
-        Map 6 
-            Map Operator Tree:
-                TableScan
-                  alias: c
-                  filterExpr: value is not null (type: boolean)
-                  Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: value is not null (type: boolean)
-                    Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
-                    Select Operator
-                      expressions: value (type: string)
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: string)
-                        null sort order: z
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: string)
-                        Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: llap
-            LLAP IO: no inputs
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
@@ -380,35 +372,13 @@ STAGE PLANS:
 #### 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)
+        Reducer 3 <- Map 5 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
         Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
-                  alias: a
-                  filterExpr: ((UDFToDouble(key) > 100.0D) and value is not null) (type: boolean)
-                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: ((UDFToDouble(key) > 100.0D) and value is not null) (type: boolean)
-                    Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-                    Select Operator
-                      expressions: key (type: string), value (type: string)
-                      outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: string)
-                        null sort order: z
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: string)
-                        Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions: _col1 (type: string)
-            Execution mode: llap
-            LLAP IO: no inputs
-        Map 5 
-            Map Operator Tree:
-                TableScan
                   alias: b
                   filterExpr: (UDFToDouble(key) > 100.0D) (type: boolean)
                   Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
@@ -427,11 +397,11 @@ STAGE PLANS:
                         Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: llap
             LLAP IO: no inputs
-        Map 6 
+        Map 5 
             Map Operator Tree:
                 TableScan
                   alias: c
-                  filterExpr: value is not null (type: boolean)
+                  filterExpr: (value is not null or ((UDFToDouble(key) > 100.0D) and value is not null)) (type: boolean)
                   Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: value is not null (type: boolean)
@@ -446,6 +416,20 @@ STAGE PLANS:
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: ((UDFToDouble(key) > 100.0D) and value is not null) (type: boolean)
+                    Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: string), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: string)
             Execution mode: llap
             LLAP IO: no inputs
         Reducer 2 
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 b80be34..d91866b 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
@@ -184,14 +184,14 @@ STAGE PLANS:
 #### 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)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: a
-                  filterExpr: (key is not null and value is not null) (type: boolean)
+                  filterExpr: ((key is not null and value is not null) or key is not null) (type: boolean)
                   Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key is not null and value is not null) (type: boolean)
@@ -207,6 +207,19 @@ STAGE PLANS:
                         Map-reduce partition columns: _col1 (type: string)
                         Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int)
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 4 
@@ -231,27 +244,6 @@ STAGE PLANS:
                         value expressions: _col0 (type: int)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 5 
-            Map Operator Tree:
-                TableScan
-                  alias: c
-                  filterExpr: key is not null (type: boolean)
-                  Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: key is not null (type: boolean)
-                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                    Select Operator
-                      expressions: key (type: int)
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: int)
-                        null sort order: z
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
@@ -319,14 +311,14 @@ STAGE PLANS:
 #### 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)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: a
-                  filterExpr: (key is not null and value is not null) (type: boolean)
+                  filterExpr: ((key is not null and value is not null) or key is not null) (type: boolean)
                   Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key is not null and value is not null) (type: boolean)
@@ -342,6 +334,19 @@ STAGE PLANS:
                         Map-reduce partition columns: _col1 (type: string)
                         Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int)
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 4 
@@ -366,27 +371,6 @@ STAGE PLANS:
                         value expressions: _col0 (type: int)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 5 
-            Map Operator Tree:
-                TableScan
-                  alias: c
-                  filterExpr: key is not null (type: boolean)
-                  Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: key is not null (type: boolean)
-                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                    Select Operator
-                      expressions: key (type: int)
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: int)
-                        null sort order: z
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
@@ -1391,14 +1375,14 @@ STAGE PLANS:
 #### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
-        Reducer 3 <- Map 4 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: tab_n10
-                  filterExpr: UDFToDouble(value) is not null (type: boolean)
+                  filterExpr: (UDFToDouble(value) is not null or UDFToDouble(key) is not null) (type: boolean)
                   Statistics: Num rows: 242 Data size: 22022 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: UDFToDouble(value) is not null (type: boolean)
@@ -1415,14 +1399,6 @@ STAGE PLANS:
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 121 Data size: 11011 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 4 
-            Map Operator Tree:
-                TableScan
-                  alias: b
-                  filterExpr: UDFToDouble(key) is not null (type: boolean)
-                  Statistics: Num rows: 242 Data size: 22990 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: UDFToDouble(key) is not null (type: boolean)
                     Statistics: Num rows: 242 Data size: 22990 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1507,14 +1483,14 @@ STAGE PLANS:
 #### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
-        Reducer 3 <- Map 4 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: tab_n10
-                  filterExpr: UDFToDouble(value) is not null (type: boolean)
+                  filterExpr: (UDFToDouble(value) is not null or UDFToDouble(key) is not null) (type: boolean)
                   Statistics: Num rows: 242 Data size: 22022 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: UDFToDouble(value) is not null (type: boolean)
@@ -1531,14 +1507,6 @@ STAGE PLANS:
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 121 Data size: 11011 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 4 
-            Map Operator Tree:
-                TableScan
-                  alias: b
-                  filterExpr: UDFToDouble(key) is not null (type: boolean)
-                  Statistics: Num rows: 242 Data size: 22990 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: UDFToDouble(key) is not null (type: boolean)
                     Statistics: Num rows: 242 Data size: 22990 Basic stats: COMPLETE Column stats: COMPLETE
@@ -2100,14 +2068,14 @@ STAGE PLANS:
 #### 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)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: a
-                  filterExpr: (key is not null and value is not null) (type: boolean)
+                  filterExpr: ((key is not null and value is not null) or key is not null) (type: boolean)
                   Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key is not null and value is not null) (type: boolean)
@@ -2123,6 +2091,19 @@ STAGE PLANS:
                         Map-reduce partition columns: _col1 (type: string)
                         Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int)
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 4 
@@ -2147,27 +2128,6 @@ STAGE PLANS:
                         value expressions: _col0 (type: int)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 5 
-            Map Operator Tree:
-                TableScan
-                  alias: c
-                  filterExpr: key is not null (type: boolean)
-                  Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: key is not null (type: boolean)
-                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                    Select Operator
-                      expressions: key (type: int)
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: int)
-                        null sort order: z
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
@@ -2236,14 +2196,14 @@ STAGE PLANS:
 #### 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)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: a
-                  filterExpr: (key is not null and value is not null) (type: boolean)
+                  filterExpr: ((key is not null and value is not null) or key is not null) (type: boolean)
                   Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key is not null and value is not null) (type: boolean)
@@ -2259,6 +2219,19 @@ STAGE PLANS:
                         Map-reduce partition columns: _col1 (type: string)
                         Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int)
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 4 
@@ -2283,27 +2256,6 @@ STAGE PLANS:
                         value expressions: _col0 (type: int)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 5 
-            Map Operator Tree:
-                TableScan
-                  alias: c
-                  filterExpr: key is not null (type: boolean)
-                  Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: key is not null (type: boolean)
-                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                    Select Operator
-                      expressions: key (type: int)
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: int)
-                        null sort order: z
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
diff --git a/ql/src/test/results/clientpositive/llap/cbo_SortUnionTransposeRule.q.out b/ql/src/test/results/clientpositive/llap/cbo_SortUnionTransposeRule.q.out
index e7a99d4..a1dbcca 100644
--- a/ql/src/test/results/clientpositive/llap/cbo_SortUnionTransposeRule.q.out
+++ b/ql/src/test/results/clientpositive/llap/cbo_SortUnionTransposeRule.q.out
@@ -913,18 +913,17 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE), Union 4 (CONTAINS)
-        Reducer 5 <- Map 1 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE)
+        Reducer 5 <- Map 1 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE)
         Reducer 6 <- Reducer 5 (CUSTOM_SIMPLE_EDGE), Union 4 (CONTAINS)
-        Reducer 8 <- Map 7 (CUSTOM_SIMPLE_EDGE)
+        Reducer 7 <- Map 1 (CUSTOM_SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: src2
-                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -947,13 +946,6 @@ STAGE PLANS:
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: string)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 7 
-            Map Operator Tree:
-                TableScan
-                  alias: src1
-                  Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: key (type: string)
                     outputColumnNames: _col0
@@ -1057,7 +1049,7 @@ STAGE PLANS:
                           input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                           output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                           serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 8 
+        Reducer 7 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
diff --git a/ql/src/test/results/clientpositive/llap/correlationoptimizer10.q.out b/ql/src/test/results/clientpositive/llap/correlationoptimizer10.q.out
index c1577e7..cc47572 100644
--- a/ql/src/test/results/clientpositive/llap/correlationoptimizer10.q.out
+++ b/ql/src/test/results/clientpositive/llap/correlationoptimizer10.q.out
@@ -807,8 +807,8 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-        Reducer 4 <- Map 3 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -831,17 +831,6 @@ STAGE PLANS:
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 55 Data size: 9790 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: string)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 3 
-            Map Operator Tree:
-                TableScan
-                  alias: x
-                  filterExpr: ((UDFToDouble(key) < 200.0D) and (UDFToDouble(key) > 180.0D)) (type: boolean)
-                  Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: ((UDFToDouble(key) < 200.0D) and (UDFToDouble(key) > 180.0D)) (type: boolean)
-                    Statistics: Num rows: 55 Data size: 4785 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: string)
                       outputColumnNames: _col0
@@ -854,7 +843,7 @@ STAGE PLANS:
                         Statistics: Num rows: 55 Data size: 4785 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 5 
+        Map 4 
             Map Operator Tree:
                 TableScan
                   alias: y
@@ -893,7 +882,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 4 
+        Reducer 3 
             Execution mode: llap
             Reduce Operator Tree:
               Merge Join Operator
@@ -1002,8 +991,8 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-        Reducer 4 <- Map 3 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -1026,17 +1015,6 @@ STAGE PLANS:
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 55 Data size: 9790 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: string)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 3 
-            Map Operator Tree:
-                TableScan
-                  alias: x
-                  filterExpr: ((UDFToDouble(key) < 200.0D) and (UDFToDouble(key) > 180.0D)) (type: boolean)
-                  Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: ((UDFToDouble(key) < 200.0D) and (UDFToDouble(key) > 180.0D)) (type: boolean)
-                    Statistics: Num rows: 55 Data size: 4785 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: string)
                       outputColumnNames: _col0
@@ -1049,7 +1027,7 @@ STAGE PLANS:
                         Statistics: Num rows: 55 Data size: 4785 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 5 
+        Map 4 
             Map Operator Tree:
                 TableScan
                   alias: y
@@ -1088,7 +1066,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 4 
+        Reducer 3 
             Execution mode: llap
             Reduce Operator Tree:
               Merge Join Operator
diff --git a/ql/src/test/results/clientpositive/llap/correlationoptimizer3.q.out b/ql/src/test/results/clientpositive/llap/correlationoptimizer3.q.out
index 37b96ad..9ba4447 100644
--- a/ql/src/test/results/clientpositive/llap/correlationoptimizer3.q.out
+++ b/ql/src/test/results/clientpositive/llap/correlationoptimizer3.q.out
@@ -27,11 +27,11 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE)
-        Reducer 3 <- Map 8 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
+        Reducer 3 <- Map 7 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
         Reducer 4 <- Reducer 3 (CUSTOM_SIMPLE_EDGE)
-        Reducer 6 <- Map 5 (SIMPLE_EDGE), Map 8 (SIMPLE_EDGE)
-        Reducer 7 <- Reducer 6 (SIMPLE_EDGE)
+        Reducer 5 <- Map 1 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE)
+        Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -54,17 +54,6 @@ STAGE PLANS:
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: string)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 5 
-            Map Operator Tree:
-                TableScan
-                  alias: x
-                  filterExpr: key is not null (type: boolean)
-                  Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: key is not null (type: boolean)
-                    Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: string)
                       outputColumnNames: _col0
@@ -77,7 +66,7 @@ STAGE PLANS:
                         Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 8 
+        Map 7 
             Map Operator Tree:
                 TableScan
                   alias: y
@@ -163,7 +152,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 6 
+        Reducer 5 
             Execution mode: llap
             Reduce Operator Tree:
               Merge Join Operator
@@ -188,7 +177,7 @@ STAGE PLANS:
                     Map-reduce partition columns: _col0 (type: string)
                     Statistics: Num rows: 16 Data size: 1504 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col1 (type: bigint)
-        Reducer 7 
+        Reducer 6 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -259,11 +248,11 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE)
-        Reducer 3 <- Map 8 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
+        Reducer 3 <- Map 7 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
         Reducer 4 <- Reducer 3 (CUSTOM_SIMPLE_EDGE)
-        Reducer 6 <- Map 5 (SIMPLE_EDGE), Map 8 (SIMPLE_EDGE)
-        Reducer 7 <- Reducer 6 (SIMPLE_EDGE)
+        Reducer 5 <- Map 1 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE)
+        Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -286,17 +275,6 @@ STAGE PLANS:
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: string)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 5 
-            Map Operator Tree:
-                TableScan
-                  alias: x
-                  filterExpr: key is not null (type: boolean)
-                  Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: key is not null (type: boolean)
-                    Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: string)
                       outputColumnNames: _col0
@@ -309,7 +287,7 @@ STAGE PLANS:
                         Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 8 
+        Map 7 
             Map Operator Tree:
                 TableScan
                   alias: y
@@ -395,7 +373,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 6 
+        Reducer 5 
             Execution mode: llap
             Reduce Operator Tree:
               Merge Join Operator
@@ -420,7 +398,7 @@ STAGE PLANS:
                     Map-reduce partition columns: _col0 (type: string)
                     Statistics: Num rows: 16 Data size: 1504 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col1 (type: bigint)
-        Reducer 7 
+        Reducer 6 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -491,10 +469,10 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Map 2 <- Map 4 (BROADCAST_EDGE)
-        Map 5 <- Reducer 3 (BROADCAST_EDGE)
+        Map 2 <- Map 1 (BROADCAST_EDGE)
+        Map 4 <- Reducer 3 (BROADCAST_EDGE)
         Reducer 3 <- Map 1 (BROADCAST_EDGE), Map 2 (SIMPLE_EDGE)
-        Reducer 6 <- Map 5 (CUSTOM_SIMPLE_EDGE)
+        Reducer 5 <- Map 4 (CUSTOM_SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -517,6 +495,16 @@ STAGE PLANS:
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: string)
+                    Select Operator
+                      expressions: key (type: string)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 2 
@@ -540,7 +528,7 @@ STAGE PLANS:
                           1 _col0 (type: string)
                         outputColumnNames: _col1
                         input vertices:
-                          1 Map 4
+                          1 Map 1
                         Statistics: Num rows: 39 Data size: 3354 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
@@ -561,27 +549,6 @@ STAGE PLANS:
         Map 4 
             Map Operator Tree:
                 TableScan
-                  alias: x
-                  filterExpr: key is not null (type: boolean)
-                  Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: key is not null (type: boolean)
-                    Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
-                    Select Operator
-                      expressions: key (type: string)
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: string)
-                        null sort order: z
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: string)
-                        Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 5 
-            Map Operator Tree:
-                TableScan
                   alias: y
                   filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
@@ -645,7 +612,7 @@ STAGE PLANS:
                     Map-reduce partition columns: _col0 (type: string)
                     Statistics: Num rows: 25 Data size: 4575 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col1 (type: string), _col3 (type: bigint)
-        Reducer 6 
+        Reducer 5 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -718,7 +685,7 @@ STAGE PLANS:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
         Reducer 4 <- Reducer 3 (CUSTOM_SIMPLE_EDGE)
-        Reducer 5 <- Map 1 (SIMPLE_EDGE), Map 8 (SIMPLE_EDGE)
+        Reducer 5 <- Map 1 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE)
         Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
@@ -769,17 +736,6 @@ STAGE PLANS:
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: string)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 8 
-            Map Operator Tree:
-                TableScan
-                  alias: x
-                  filterExpr: key is not null (type: boolean)
-                  Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: key is not null (type: boolean)
-                    Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: string)
                       outputColumnNames: _col0
@@ -950,7 +906,7 @@ STAGE PLANS:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
         Reducer 4 <- Reducer 3 (CUSTOM_SIMPLE_EDGE)
-        Reducer 5 <- Map 1 (SIMPLE_EDGE), Map 8 (SIMPLE_EDGE)
+        Reducer 5 <- Map 1 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE)
         Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
@@ -1001,17 +957,6 @@ STAGE PLANS:
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: string)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 8 
-            Map Operator Tree:
-                TableScan
-                  alias: x
-                  filterExpr: key is not null (type: boolean)
-                  Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: key is not null (type: boolean)
-                    Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: string)
                       outputColumnNames: _col0
@@ -1180,7 +1125,7 @@ STAGE PLANS:
 #### A masked pattern was here ####
       Edges:
         Map 1 <- Map 2 (BROADCAST_EDGE)
-        Map 3 <- Map 6 (BROADCAST_EDGE)
+        Map 3 <- Map 2 (BROADCAST_EDGE)
         Reducer 4 <- Map 1 (BROADCAST_EDGE), Map 3 (SIMPLE_EDGE)
         Reducer 5 <- Reducer 4 (CUSTOM_SIMPLE_EDGE)
 #### A masked pattern was here ####
@@ -1237,6 +1182,16 @@ STAGE PLANS:
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: string)
+                    Select Operator
+                      expressions: key (type: string)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 3 
@@ -1260,7 +1215,7 @@ STAGE PLANS:
                           1 _col0 (type: string)
                         outputColumnNames: _col1
                         input vertices:
-                          1 Map 6
+                          1 Map 2
                         Statistics: Num rows: 39 Data size: 3354 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
@@ -1278,27 +1233,6 @@ STAGE PLANS:
                             value expressions: _col1 (type: bigint)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 6 
-            Map Operator Tree:
-                TableScan
-                  alias: x
-                  filterExpr: key is not null (type: boolean)
-                  Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: key is not null (type: boolean)
-                    Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
-                    Select Operator
-                      expressions: key (type: string)
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: string)
-                        null sort order: z
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: string)
-                        Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
         Reducer 4 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
diff --git a/ql/src/test/results/clientpositive/llap/correlationoptimizer6.q.out b/ql/src/test/results/clientpositive/llap/correlationoptimizer6.q.out
index 71fefd9..a1ec998 100644
--- a/ql/src/test/results/clientpositive/llap/correlationoptimizer6.q.out
+++ b/ql/src/test/results/clientpositive/llap/correlationoptimizer6.q.out
@@ -2940,7 +2940,7 @@ STAGE PLANS:
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
-        Reducer 4 <- Map 7 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+        Reducer 4 <- Map 1 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -2966,6 +2966,17 @@ STAGE PLANS:
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 250 Data size: 23750 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: bigint)
+                    Select Operator
+                      expressions: key (type: string), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 5 
@@ -2993,28 +3004,6 @@ STAGE PLANS:
                         value expressions: _col1 (type: bigint)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 7 
-            Map Operator Tree:
-                TableScan
-                  alias: z
-                  filterExpr: key is not null (type: boolean)
-                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: key is not null (type: boolean)
-                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                    Select Operator
-                      expressions: key (type: string), value (type: string)
-                      outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: string)
-                        null sort order: z
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: string)
-                        Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions: _col1 (type: string)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
         Reducer 2 
             Reduce Operator Tree:
               Group By Operator
@@ -3193,7 +3182,7 @@ STAGE PLANS:
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
-        Reducer 4 <- Map 7 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+        Reducer 4 <- Map 1 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -3219,6 +3208,17 @@ STAGE PLANS:
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 250 Data size: 23750 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: bigint)
+                    Select Operator
+                      expressions: key (type: string), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 5 
@@ -3246,28 +3246,6 @@ STAGE PLANS:
                         value expressions: _col1 (type: bigint)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 7 
-            Map Operator Tree:
-                TableScan
-                  alias: z
-                  filterExpr: key is not null (type: boolean)
-                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: key is not null (type: boolean)
-                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                    Select Operator
-                      expressions: key (type: string), value (type: string)
-                      outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: string)
-                        null sort order: z
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: string)
-                        Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions: _col1 (type: string)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
         Reducer 2 
             Reduce Operator Tree:
               Group By Operator
diff --git a/ql/src/test/results/clientpositive/llap/correlationoptimizer7.q.out b/ql/src/test/results/clientpositive/llap/correlationoptimizer7.q.out
index 9193efb..2af6986 100644
--- a/ql/src/test/results/clientpositive/llap/correlationoptimizer7.q.out
+++ b/ql/src/test/results/clientpositive/llap/correlationoptimizer7.q.out
@@ -30,7 +30,7 @@ STAGE PLANS:
 #### A masked pattern was here ####
       Edges:
         Map 1 <- Map 3 (BROADCAST_EDGE)
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (BROADCAST_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 3 (BROADCAST_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -91,17 +91,6 @@ STAGE PLANS:
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 4 
-            Map Operator Tree:
-                TableScan
-                  alias: yy
-                  filterExpr: key is not null (type: boolean)
-                  Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: key is not null (type: boolean)
-                    Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: string), value (type: string)
                       outputColumnNames: _col0, _col1
@@ -132,7 +121,7 @@ STAGE PLANS:
                     1 _col0 (type: string)
                   outputColumnNames: _col0, _col1, _col2, _col3
                   input vertices:
-                    1 Map 4
+                    1 Map 3
                   Statistics: Num rows: 3 Data size: 810 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
@@ -215,7 +204,7 @@ STAGE PLANS:
 #### A masked pattern was here ####
       Edges:
         Map 1 <- Map 3 (BROADCAST_EDGE)
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (BROADCAST_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 3 (BROADCAST_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -276,17 +265,6 @@ STAGE PLANS:
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 4 
-            Map Operator Tree:
-                TableScan
-                  alias: yy
-                  filterExpr: key is not null (type: boolean)
-                  Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: key is not null (type: boolean)
-                    Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: string), value (type: string)
                       outputColumnNames: _col0, _col1
@@ -317,7 +295,7 @@ STAGE PLANS:
                     1 _col0 (type: string)
                   outputColumnNames: _col0, _col1, _col2, _col3
                   input vertices:
-                    1 Map 4
+                    1 Map 3
                   Statistics: Num rows: 3 Data size: 810 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
@@ -400,7 +378,7 @@ STAGE PLANS:
 #### A masked pattern was here ####
       Edges:
         Map 1 <- Map 3 (BROADCAST_EDGE)
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (BROADCAST_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 3 (BROADCAST_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -461,17 +439,6 @@ STAGE PLANS:
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 4 
-            Map Operator Tree:
-                TableScan
-                  alias: yy
-                  filterExpr: key is not null (type: boolean)
-                  Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: key is not null (type: boolean)
-                    Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: string), value (type: string)
                       outputColumnNames: _col0, _col1
@@ -502,7 +469,7 @@ STAGE PLANS:
                     1 _col0 (type: string)
                   outputColumnNames: _col0, _col1, _col2, _col3
                   input vertices:
-                    1 Map 4
+                    1 Map 3
                   Statistics: Num rows: 3 Data size: 810 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
@@ -585,7 +552,7 @@ STAGE PLANS:
 #### A masked pattern was here ####
       Edges:
         Map 1 <- Map 3 (BROADCAST_EDGE)
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (BROADCAST_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 3 (BROADCAST_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -646,17 +613,6 @@ STAGE PLANS:
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 4 
-            Map Operator Tree:
-                TableScan
-                  alias: yy
-                  filterExpr: key is not null (type: boolean)
-                  Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: key is not null (type: boolean)
-                    Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: string), value (type: string)
                       outputColumnNames: _col0, _col1
@@ -687,7 +643,7 @@ STAGE PLANS:
                     1 _col0 (type: string)
                   outputColumnNames: _col0, _col1, _col2, _col3
                   input vertices:
-                    1 Map 4
+                    1 Map 3
                   Statistics: Num rows: 3 Data size: 810 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
diff --git a/ql/src/test/results/clientpositive/llap/explainanalyze_2.q.out b/ql/src/test/results/clientpositive/llap/explainanalyze_2.q.out
index 343aad1..1ef44e53 100644
--- a/ql/src/test/results/clientpositive/llap/explainanalyze_2.q.out
+++ b/ql/src/test/results/clientpositive/llap/explainanalyze_2.q.out
@@ -669,7 +669,7 @@ Plan optimized by CBO.
 
 Vertex dependency in root stage
 Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
-Reducer 3 <- Map 5 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+Reducer 3 <- Map 1 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
@@ -679,15 +679,15 @@ Stage-0
       File Output Operator [FS_16]
         Merge Join Operator [MERGEJOIN_46] (rows=633/1166 width=95)
           Conds:RS_12._col0=RS_13._col0(Inner),Output:["_col0","_col1"]
-        <-Map 5 [SIMPLE_EDGE] llap
+        <-Map 1 [SIMPLE_EDGE] llap
           SHUFFLE [RS_13]
             PartitionCols:_col0
             Select Operator [SEL_8] (rows=242/242 width=4)
               Output:["_col0"]
               Filter Operator [FIL_24] (rows=242/242 width=4)
                 predicate:key is not null
-                TableScan [TS_6] (rows=242/242 width=4)
-                  default@tab_n6,s3,Tbl:COMPLETE,Col:COMPLETE,Output:["key"]
+                TableScan [TS_0] (rows=242/242 width=95)
+                  default@tab_n6,s1,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
         <-Reducer 2 [SIMPLE_EDGE] llap
           SHUFFLE [RS_12]
             PartitionCols:_col0
@@ -700,8 +700,7 @@ Stage-0
                   Output:["_col0","_col1"]
                   Filter Operator [FIL_22] (rows=242/242 width=95)
                     predicate:(key is not null and value is not null)
-                    TableScan [TS_0] (rows=242/242 width=95)
-                      default@tab_n6,s1,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
+                     Please refer to the previous TableScan [TS_0]
             <-Map 4 [SIMPLE_EDGE] llap
               SHUFFLE [RS_10]
                 PartitionCols:_col0
@@ -800,7 +799,7 @@ Plan optimized by CBO.
 
 Vertex dependency in root stage
 Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
-Reducer 3 <- Map 5 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+Reducer 3 <- Map 4 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
@@ -810,20 +809,28 @@ Stage-0
       File Output Operator [FS_16]
         Merge Join Operator [MERGEJOIN_46] (rows=633/1166 width=95)
           Conds:RS_12._col0=RS_13._col0(Inner),Output:["_col0","_col1"]
-        <-Map 5 [SIMPLE_EDGE] llap
+        <-Map 4 [SIMPLE_EDGE] llap
           SHUFFLE [RS_13]
             PartitionCols:_col0
             Select Operator [SEL_8] (rows=242/242 width=4)
               Output:["_col0"]
               Filter Operator [FIL_24] (rows=242/242 width=4)
                 predicate:key is not null
-                TableScan [TS_6] (rows=242/242 width=4)
-                  default@tab2_n3,s3,Tbl:COMPLETE,Col:COMPLETE,Output:["key"]
+                TableScan [TS_3] (rows=242/242 width=91)
+                  default@tab2_n3,s2,Tbl:COMPLETE,Col:COMPLETE,Output:["value","key"]
         <-Reducer 2 [SIMPLE_EDGE] llap
           SHUFFLE [RS_12]
             PartitionCols:_col0
             Merge Join Operator [MERGEJOIN_45] (rows=382/480 width=95)
               Conds:RS_9._col1=RS_10._col0(Inner),Output:["_col0","_col1"]
+            <-Map 4 [SIMPLE_EDGE] llap
+              SHUFFLE [RS_10]
+                PartitionCols:_col0
+                Select Operator [SEL_5] (rows=242/242 width=91)
+                  Output:["_col0"]
+                  Filter Operator [FIL_23] (rows=242/242 width=91)
+                    predicate:value is not null
+                     Please refer to the previous TableScan [TS_3]
             <-Map 1 [SIMPLE_EDGE] llap
               SHUFFLE [RS_9]
                 PartitionCols:_col1
@@ -833,15 +840,6 @@ Stage-0
                     predicate:(key is not null and value is not null)
                     TableScan [TS_0] (rows=242/242 width=95)
                       default@tab_n6,s1,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
-            <-Map 4 [SIMPLE_EDGE] llap
-              SHUFFLE [RS_10]
-                PartitionCols:_col0
-                Select Operator [SEL_5] (rows=242/242 width=91)
-                  Output:["_col0"]
-                  Filter Operator [FIL_23] (rows=242/242 width=91)
-                    predicate:value is not null
-                    TableScan [TS_3] (rows=242/242 width=91)
-                      default@tab2_n3,s2,Tbl:COMPLETE,Col:COMPLETE,Output:["value"]
 
 PREHOOK: query: select count(*) from (select s1.key as key, s1.value as value from tab_n6 s1 join tab_n6 s3 on s1.key=s3.key
 UNION  ALL
@@ -988,10 +986,10 @@ POSTHOOK: Input: default@tab_part_n7@ds=2008-04-08
 Plan optimized by CBO.
 
 Vertex dependency in root stage
-Map 9 <- Union 4 (CONTAINS)
+Map 8 <- Union 4 (CONTAINS)
 Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE)
-Reducer 3 <- Map 8 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE), Union 4 (CONTAINS)
-Reducer 5 <- Map 10 (SIMPLE_EDGE), Union 4 (SIMPLE_EDGE)
+Reducer 3 <- Map 1 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE), Union 4 (CONTAINS)
+Reducer 5 <- Map 9 (SIMPLE_EDGE), Union 4 (SIMPLE_EDGE)
 Reducer 6 <- Reducer 5 (CUSTOM_SIMPLE_EDGE)
 
 Stage-0
@@ -1006,7 +1004,7 @@ Stage-0
           PARTITION_ONLY_SHUFFLE [RS_28]
             Merge Join Operator [MERGEJOIN_81] (rows=1443/3768 width=8)
               Conds:Union 4._col0=RS_25._col0(Inner)
-            <-Map 10 [SIMPLE_EDGE] llap
+            <-Map 9 [SIMPLE_EDGE] llap
               SHUFFLE [RS_25]
                 PartitionCols:_col0
                 Select Operator [SEL_23] (rows=500/500 width=4)
@@ -1016,7 +1014,7 @@ Stage-0
                     TableScan [TS_21] (rows=500/500 width=4)
                       default@tab_part_n7,b_n10,Tbl:COMPLETE,Col:COMPLETE,Output:["key"]
             <-Union 4 [SIMPLE_EDGE]
-              <-Map 9 [CONTAINS] llap
+              <-Map 8 [CONTAINS] llap
                 Reduce Output Operator [RS_89]
                   PartitionCols:_col0
                   Select Operator [SEL_87] (rows=242/242 width=4)
@@ -1030,15 +1028,15 @@ Stage-0
                   PartitionCols:_col0
                   Merge Join Operator [MERGEJOIN_82] (rows=633/1166 width=4)
                     Conds:RS_12._col0=RS_13._col0(Inner),Output:["_col0"]
-                  <-Map 8 [SIMPLE_EDGE] llap
+                  <-Map 1 [SIMPLE_EDGE] llap
                     SHUFFLE [RS_13]
                       PartitionCols:_col0
                       Select Operator [SEL_8] (rows=242/242 width=4)
                         Output:["_col0"]
                         Filter Operator [FIL_44] (rows=242/242 width=4)
                           predicate:key is not null
-                          TableScan [TS_6] (rows=242/242 width=4)
-                            default@tab_n6,s3,Tbl:COMPLETE,Col:COMPLETE,Output:["key"]
+                          TableScan [TS_0] (rows=242/242 width=95)
+                            default@tab_n6,s1,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
                   <-Reducer 2 [SIMPLE_EDGE] llap
                     SHUFFLE [RS_12]
                       PartitionCols:_col0
@@ -1051,8 +1049,7 @@ Stage-0
                             Output:["_col0","_col1"]
                             Filter Operator [FIL_42] (rows=242/242 width=95)
                               predicate:(key is not null and value is not null)
-                              TableScan [TS_0] (rows=242/242 width=95)
-                                default@tab_n6,s1,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
+                               Please refer to the previous TableScan [TS_0]
                       <-Map 7 [SIMPLE_EDGE] llap
                         SHUFFLE [RS_10]
                           PartitionCols:_col0
diff --git a/ql/src/test/results/clientpositive/llap/explainuser_1.q.out b/ql/src/test/results/clientpositive/llap/explainuser_1.q.out
index e26de03..548fc65 100644
--- a/ql/src/test/results/clientpositive/llap/explainuser_1.q.out
+++ b/ql/src/test/results/clientpositive/llap/explainuser_1.q.out
@@ -2379,7 +2379,7 @@ Plan optimized by CBO.
 
 Vertex dependency in root stage
 Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
-Reducer 3 <- Map 5 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+Reducer 3 <- Map 1 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
@@ -2391,7 +2391,7 @@ Stage-0
           Output:["_col0","_col1"]
           Merge Join Operator [MERGEJOIN_51] (rows=1 width=8)
             Conds:RS_17._col1, _col4=RS_18._col0, _col1(Left Semi),Output:["_col0","_col3"]
-          <-Map 5 [SIMPLE_EDGE] llap
+          <-Map 1 [SIMPLE_EDGE] llap
             SHUFFLE [RS_18]
               PartitionCols:_col0, _col1
               Group By Operator [GBY_16] (rows=1 width=8)
@@ -2400,8 +2400,8 @@ Stage-0
                   Output:["_col0","_col1"]
                   Filter Operator [FIL_29] (rows=2 width=96)
                     predicate:((l_linenumber = 1) and (l_shipmode = 'AIR') and l_orderkey is not null)
-                    TableScan [TS_12] (rows=100 width=96)
-                      default@lineitem,lineitem,Tbl:COMPLETE,Col:COMPLETE,Output:["l_orderkey","l_linenumber","l_shipmode"]
+                    TableScan [TS_0] (rows=100 width=16)
+                      default@lineitem,li,Tbl:COMPLETE,Col:COMPLETE,Output:["l_orderkey","l_partkey","l_suppkey","l_linenumber","l_shipmode"]
           <-Reducer 2 [SIMPLE_EDGE] llap
             SHUFFLE [RS_17]
               PartitionCols:_col1, _col4
@@ -2416,8 +2416,7 @@ Stage-0
                       Output:["_col0","_col1","_col2","_col3"]
                       Filter Operator [FIL_27] (rows=14 width=16)
                         predicate:((l_linenumber = 1) and l_partkey is not null and l_orderkey is not null)
-                        TableScan [TS_0] (rows=100 width=16)
-                          default@lineitem,li,Tbl:COMPLETE,Col:COMPLETE,Output:["l_orderkey","l_partkey","l_suppkey","l_linenumber"]
+                         Please refer to the previous TableScan [TS_0]
                 <-Map 4 [SIMPLE_EDGE] llap
                   SHUFFLE [RS_9]
                     PartitionCols:_col0
@@ -2450,7 +2449,7 @@ Plan optimized by CBO.
 
 Vertex dependency in root stage
 Reducer 2 <- Map 1 (SIMPLE_EDGE)
-Reducer 3 <- Map 7 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+Reducer 3 <- Map 1 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
 Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
 Reducer 5 <- Reducer 4 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
 Reducer 6 <- Map 1 (SIMPLE_EDGE)
@@ -2477,15 +2476,15 @@ Stage-0
                     Output:["_col0","_col1","_col2"],aggregations:["count()"],keys:_col1, _col2
                     Merge Join Operator [MERGEJOIN_55] (rows=131 width=178)
                       Conds:RS_10._col0=RS_11._col0(Inner),Output:["_col1","_col2"]
-                    <-Map 7 [SIMPLE_EDGE] llap
+                    <-Map 1 [SIMPLE_EDGE] llap
                       SHUFFLE [RS_11]
                         PartitionCols:_col0
                         Select Operator [SEL_9] (rows=166 width=178)
                           Output:["_col0","_col1"]
                           Filter Operator [FIL_41] (rows=166 width=178)
                             predicate:(key > '8')
-                            TableScan [TS_7] (rows=500 width=178)
-                              default@src_cbo,b,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
+                            TableScan [TS_0] (rows=500 width=87)
+                              default@src_cbo,src_cbo,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
                     <-Reducer 2 [SIMPLE_EDGE] llap
                       SHUFFLE [RS_10]
                         PartitionCols:_col0
@@ -2498,8 +2497,7 @@ Stage-0
                               Output:["_col0"],keys:key
                               Filter Operator [FIL_40] (rows=166 width=87)
                                 predicate:(key > '8')
-                                TableScan [TS_0] (rows=500 width=87)
-                                  default@src_cbo,src_cbo,Tbl:COMPLETE,Col:COMPLETE,Output:["key"]
+                                 Please refer to the previous TableScan [TS_0]
         <-Reducer 6 [SIMPLE_EDGE] llap
           SHUFFLE [RS_30]
             PartitionCols:_col0
@@ -2542,8 +2540,8 @@ Plan optimized by CBO.
 
 Vertex dependency in root stage
 Reducer 2 <- Map 1 (SIMPLE_EDGE)
-Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
-Reducer 5 <- Map 4 (SIMPLE_EDGE)
+Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
+Reducer 4 <- Map 1 (SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
@@ -2569,7 +2567,7 @@ Stage-0
                       predicate:p_name is not null
                       TableScan [TS_0] (rows=26 width=223)
                         default@part,part,Tbl:COMPLETE,Col:COMPLETE,Output:["p_name","p_mfgr","p_size"]
-        <-Reducer 5 [SIMPLE_EDGE] llap
+        <-Reducer 4 [SIMPLE_EDGE] llap
           SHUFFLE [RS_18]
             PartitionCols:_col0
             Group By Operator [GBY_16] (rows=13 width=184)
@@ -2582,11 +2580,10 @@ Stage-0
                     Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col5 ASC NULLS LAST","partition by:":"_col2"}]
                     Select Operator [SEL_9] (rows=26 width=491)
                       Output:["_col1","_col2","_col5"]
-                    <-Map 4 [SIMPLE_EDGE] llap
+                    <-Map 1 [SIMPLE_EDGE] llap
                       SHUFFLE [RS_8]
                         PartitionCols:p_mfgr
-                        TableScan [TS_7] (rows=26 width=223)
-                          default@part,part,Tbl:COMPLETE,Col:COMPLETE,Output:["p_mfgr","p_name","p_size"]
+                         Please refer to the previous TableScan [TS_0]
 
 PREHOOK: query: explain select * 
 from src_cbo 
@@ -2609,11 +2606,11 @@ POSTHOOK: Input: default@src_cbo
 Plan optimized by CBO.
 
 Vertex dependency in root stage
-Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
-Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE), Reducer 7 (CUSTOM_SIMPLE_EDGE)
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
+Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE), Reducer 6 (CUSTOM_SIMPLE_EDGE)
 Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
-Reducer 6 <- Map 5 (SIMPLE_EDGE)
-Reducer 7 <- Map 5 (CUSTOM_SIMPLE_EDGE)
+Reducer 5 <- Map 1 (SIMPLE_EDGE)
+Reducer 6 <- Map 1 (CUSTOM_SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
@@ -2638,39 +2635,38 @@ Stage-0
                       Merge Join Operator [MERGEJOIN_36] (rows=631 width=178)
                         Conds:RS_16._col0=RS_17._col0(Left Outer),Output:["_col0","_col1","_col3"]
                       <-Map 1 [SIMPLE_EDGE] llap
-                        SHUFFLE [RS_16]
+                        PARTITION_ONLY_SHUFFLE [RS_16]
                           PartitionCols:_col0
                           Select Operator [SEL_1] (rows=500 width=178)
                             Output:["_col0","_col1"]
                             TableScan [TS_0] (rows=500 width=178)
                               default@src_cbo,src_cbo,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
-                      <-Reducer 6 [SIMPLE_EDGE] llap
+                      <-Reducer 5 [SIMPLE_EDGE] llap
                         SHUFFLE [RS_17]
                           PartitionCols:_col0
                           Select Operator [SEL_8] (rows=83 width=91)
                             Output:["_col0","_col1"]
                             Group By Operator [GBY_7] (rows=83 width=87)
                               Output:["_col0"],keys:KEY._col0
-                            <-Map 5 [SIMPLE_EDGE] llap
+                            <-Map 1 [SIMPLE_EDGE] llap
                               PARTITION_ONLY_SHUFFLE [RS_6]
                                 PartitionCols:_col0
                                 Group By Operator [GBY_5] (rows=83 width=87)
                                   Output:["_col0"],keys:key
                                   Filter Operator [FIL_29] (rows=166 width=87)
                                     predicate:(key > '2')
-                                    TableScan [TS_2] (rows=500 width=87)
-                                      default@src_cbo,s1,Tbl:COMPLETE,Col:COMPLETE,Output:["key"]
-                  <-Reducer 7 [CUSTOM_SIMPLE_EDGE] llap
+                                     Please refer to the previous TableScan [TS_0]
+                  <-Reducer 6 [CUSTOM_SIMPLE_EDGE] llap
                     PARTITION_ONLY_SHUFFLE [RS_20]
                       Group By Operator [GBY_14] (rows=1 width=16)
                         Output:["_col0","_col1"],aggregations:["count(VALUE._col0)","count(VALUE._col1)"]
-                      <-Map 5 [CUSTOM_SIMPLE_EDGE] llap
+                      <-Map 1 [CUSTOM_SIMPLE_EDGE] llap
                         PARTITION_ONLY_SHUFFLE [RS_13]
                           Group By Operator [GBY_12] (rows=1 width=16)
                             Output:["_col0","_col1"],aggregations:["count()","count(key)"]
                             Filter Operator [FIL_30] (rows=166 width=87)
                               predicate:(key > '2')
-                               Please refer to the previous TableScan [TS_2]
+                               Please refer to the previous TableScan [TS_0]
 
 PREHOOK: query: explain select p_mfgr, b.p_name, p_size 
 from part b 
@@ -2780,10 +2776,10 @@ POSTHOOK: Input: default@part
 Plan optimized by CBO.
 
 Vertex dependency in root stage
-Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
-Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE), Reducer 6 (CUSTOM_SIMPLE_EDGE)
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
+Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE), Reducer 5 (CUSTOM_SIMPLE_EDGE)
 Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
-Reducer 6 <- Map 5 (CUSTOM_SIMPLE_EDGE)
+Reducer 5 <- Map 1 (CUSTOM_SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
@@ -2803,7 +2799,7 @@ Stage-0
                   Output:["_col0","_col1","_col2","_col3","_col5"]
                   Merge Join Operator [MERGEJOIN_40] (rows=27 width=141)
                     Conds:(Inner),Output:["_col0","_col1","_col3","_col4","_col5"]
-                  <-Reducer 6 [CUSTOM_SIMPLE_EDGE] llap
+                  <-Reducer 5 [CUSTOM_SIMPLE_EDGE] llap
                     PARTITION_ONLY_SHUFFLE [RS_27]
                       Group By Operator [GBY_21] (rows=1 width=16)
                         Output:["_col0","_col1"],aggregations:["count()","count(_col0)"]
@@ -2811,19 +2807,25 @@ Stage-0
                           Output:["_col0"]
                           Group By Operator [GBY_7] (rows=1 width=16)
                             Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)"]
-                          <-Map 5 [CUSTOM_SIMPLE_EDGE] llap
+                          <-Map 1 [CUSTOM_SIMPLE_EDGE] llap
                             PARTITION_ONLY_SHUFFLE [RS_6]
                               Group By Operator [GBY_5] (rows=1 width=16)
                                 Output:["_col0","_col1"],aggregations:["sum(p_size)","count(p_size)"]
                                 Filter Operator [FIL_37] (rows=5 width=4)
                                   predicate:(p_size < 10)
-                                  TableScan [TS_2] (rows=26 width=4)
-                                    default@part,part,Tbl:COMPLETE,Col:COMPLETE,Output:["p_size"]
+                                  TableScan [TS_0] (rows=26 width=125)
+                                    default@part,part,Tbl:COMPLETE,Col:COMPLETE,Output:["p_name","p_size"]
                   <-Reducer 2 [CUSTOM_SIMPLE_EDGE] llap
                     PARTITION_ONLY_SHUFFLE [RS_26]
                       Merge Join Operator [MERGEJOIN_39] (rows=27 width=125)
                         Conds:RS_23.UDFToDouble(_col1)=RS_24._col0(Left Outer),Output:["_col0","_col1","_col3"]
-                      <-Reducer 6 [SIMPLE_EDGE] llap
+                      <-Map 1 [SIMPLE_EDGE] llap
+                        PARTITION_ONLY_SHUFFLE [RS_23]
+                          PartitionCols:UDFToDouble(_col1)
+                          Select Operator [SEL_1] (rows=26 width=125)
+                            Output:["_col0","_col1"]
+                             Please refer to the previous TableScan [TS_0]
+                      <-Reducer 5 [SIMPLE_EDGE] llap
                         PARTITION_ONLY_SHUFFLE [RS_24]
                           PartitionCols:_col0
                           Select Operator [SEL_10] (rows=1 width=12)
@@ -2831,13 +2833,6 @@ Stage-0
                             Filter Operator [FIL_9] (rows=1 width=16)
                               predicate:(_col1 is not null and UDFToDouble(_col0) is not null)
                                Please refer to the previous Group By Operator [GBY_7]
-                      <-Map 1 [SIMPLE_EDGE] llap
-                        SHUFFLE [RS_23]
-                          PartitionCols:UDFToDouble(_col1)
-                          Select Operator [SEL_1] (rows=26 width=125)
-                            Output:["_col0","_col1"]
-                            TableScan [TS_0] (rows=26 width=125)
-                              default@part,part,Tbl:COMPLETE,Col:COMPLETE,Output:["p_name","p_size"]
 
 PREHOOK: query: explain select b.p_mfgr, min(p_retailprice) 
 from part b 
diff --git a/ql/src/test/results/clientpositive/llap/filter_cond_pushdown.q.out b/ql/src/test/results/clientpositive/llap/filter_cond_pushdown.q.out
index 91e27f1..4940091 100644
--- a/ql/src/test/results/clientpositive/llap/filter_cond_pushdown.q.out
+++ b/ql/src/test/results/clientpositive/llap/filter_cond_pushdown.q.out
@@ -22,34 +22,12 @@ STAGE PLANS:
 #### 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)
+        Reducer 3 <- Map 4 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
-                  alias: m
-                  filterExpr: ((value <> '') and key is not null) (type: boolean)
-                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: ((value <> '') and key is not null) (type: boolean)
-                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                    Select Operator
-                      expressions: key (type: string), value (type: string), (value = '2008-04-08') (type: boolean)
-                      outputColumnNames: _col0, _col1, _col2
-                      Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: string)
-                        null sort order: z
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: string)
-                        Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions: _col1 (type: string), _col2 (type: boolean)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 4 
-            Map Operator Tree:
-                TableScan
                   alias: f
                   filterExpr: ((value) IN ('2008-04-08', '2008-04-09') and key is not null) (type: boolean)
                   Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
@@ -69,11 +47,11 @@ STAGE PLANS:
                         value expressions: _col1 (type: boolean), _col2 (type: boolean)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 5 
+        Map 4 
             Map Operator Tree:
                 TableScan
                   alias: g
-                  filterExpr: (value <> '') (type: boolean)
+                  filterExpr: ((value <> '') or ((value <> '') and key is not null)) (type: boolean)
                   Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (value <> '') (type: boolean)
@@ -88,6 +66,20 @@ STAGE PLANS:
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: ((value <> '') and key is not null) (type: boolean)
+                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: string), value (type: string), (value = '2008-04-08') (type: boolean)
+                      outputColumnNames: _col0, _col1, _col2
+                      Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: string), _col2 (type: boolean)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
@@ -162,34 +154,12 @@ STAGE PLANS:
 #### 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)
+        Reducer 3 <- Map 4 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
-                  alias: m
-                  filterExpr: ((value <> '') and key is not null) (type: boolean)
-                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: ((value <> '') and key is not null) (type: boolean)
-                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                    Select Operator
-                      expressions: key (type: string), value (type: string), (value = '2008-04-08') (type: boolean)
-                      outputColumnNames: _col0, _col1, _col2
-                      Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: string)
-                        null sort order: z
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: string)
-                        Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions: _col1 (type: string), _col2 (type: boolean)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 4 
-            Map Operator Tree:
-                TableScan
                   alias: f
                   filterExpr: ((value) IN ('2008-04-08', '2008-04-10', '2008-04-09') and key is not null) (type: boolean)
                   Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
@@ -209,11 +179,11 @@ STAGE PLANS:
                         value expressions: _col1 (type: boolean), _col2 (type: boolean)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 5 
+        Map 4 
             Map Operator Tree:
                 TableScan
                   alias: g
-                  filterExpr: (value <> '') (type: boolean)
+                  filterExpr: ((value <> '') or ((value <> '') and key is not null)) (type: boolean)
                   Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (value <> '') (type: boolean)
@@ -228,6 +198,20 @@ STAGE PLANS:
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: ((value <> '') and key is not null) (type: boolean)
+                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: string), value (type: string), (value = '2008-04-08') (type: boolean)
+                      outputColumnNames: _col0, _col1, _col2
+                      Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: string), _col2 (type: boolean)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
diff --git a/ql/src/test/results/clientpositive/llap/filter_join_breaktask.q.out b/ql/src/test/results/clientpositive/llap/filter_join_breaktask.q.out
index 4956d54..5cd4156 100644
--- a/ql/src/test/results/clientpositive/llap/filter_join_breaktask.q.out
+++ b/ql/src/test/results/clientpositive/llap/filter_join_breaktask.q.out
@@ -54,80 +54,12 @@ STAGE PLANS:
 #### 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)
+        Reducer 3 <- Map 4 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
-                  alias: f
-                  filterExpr: key is not null (type: boolean)
-                  Statistics: Num rows: 25 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
-                  GatherStats: false
-                  Filter Operator
-                    isSamplingPred: false
-                    predicate: key is not null (type: boolean)
-                    Statistics: Num rows: 15 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                    Select Operator
-                      expressions: key (type: int)
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 15 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        bucketingVersion: 2
-                        key expressions: _col0 (type: int)
-                        null sort order: z
-                        numBuckets: -1
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 15 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                        tag: 0
-                        auto parallelism: true
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-            Path -> Alias:
-#### A masked pattern was here ####
-            Path -> Partition:
-#### A masked pattern was here ####
-                Partition
-                  base file name: ds=2008-04-08
-                  input format: org.apache.hadoop.mapred.TextInputFormat
-                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                  partition values:
-                    ds 2008-04-08
-                  properties:
-                    column.name.delimiter ,
-                    columns key,value
-                    columns.types int:string
-#### A masked pattern was here ####
-                    name default.filter_join_breaktask
-                    partition_columns ds
-                    partition_columns.types string
-                    serialization.format 1
-                    serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                
-                    input format: org.apache.hadoop.mapred.TextInputFormat
-                    output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                    properties:
-                      bucketing_version 2
-                      column.name.delimiter ,
-                      columns key,value
-                      columns.comments 
-                      columns.types int:string
-#### A masked pattern was here ####
-                      name default.filter_join_breaktask
-                      partition_columns ds
-                      partition_columns.types string
-                      serialization.format 1
-                      serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                    name: default.filter_join_breaktask
-                  name: default.filter_join_breaktask
-            Truncated Path -> Alias:
-              /filter_join_breaktask/ds=2008-04-08 [f]
-        Map 4 
-            Map Operator Tree:
-                TableScan
                   alias: m
                   filterExpr: (key is not null and (value <> '')) (type: boolean)
                   Statistics: Num rows: 25 Data size: 2289 Basic stats: COMPLETE Column stats: COMPLETE
@@ -194,11 +126,11 @@ STAGE PLANS:
                   name: default.filter_join_breaktask
             Truncated Path -> Alias:
               /filter_join_breaktask/ds=2008-04-08 [m]
-        Map 5 
+        Map 4 
             Map Operator Tree:
                 TableScan
                   alias: g
-                  filterExpr: (value <> '') (type: boolean)
+                  filterExpr: ((value <> '') or key is not null) (type: boolean)
                   Statistics: Num rows: 25 Data size: 2225 Basic stats: COMPLETE Column stats: COMPLETE
                   GatherStats: false
                   Filter Operator
@@ -219,6 +151,24 @@ STAGE PLANS:
                         Statistics: Num rows: 25 Data size: 2225 Basic stats: COMPLETE Column stats: COMPLETE
                         tag: 1
                         auto parallelism: true
+                  Filter Operator
+                    isSamplingPred: false
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 15 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 15 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        bucketingVersion: 2
+                        key expressions: _col0 (type: int)
+                        null sort order: z
+                        numBuckets: -1
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 15 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+                        tag: 0
+                        auto parallelism: true
             Execution mode: vectorized, llap
             LLAP IO: no inputs
             Path -> Alias:
diff --git a/ql/src/test/results/clientpositive/llap/groupby_sort_1_23.q.out b/ql/src/test/results/clientpositive/llap/groupby_sort_1_23.q.out
index ccf2c15..0a6d6b4 100644
--- a/ql/src/test/results/clientpositive/llap/groupby_sort_1_23.q.out
+++ b/ql/src/test/results/clientpositive/llap/groupby_sort_1_23.q.out
@@ -3185,8 +3185,8 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-        Reducer 4 <- Map 3 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -3217,61 +3217,6 @@ STAGE PLANS:
                         tag: 0
                         value expressions: _col1 (type: bigint)
                         auto parallelism: true
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-            Path -> Alias:
-#### A masked pattern was here ####
-            Path -> Partition:
-#### A masked pattern was here ####
-                Partition
-                  base file name: t1_n80
-                  input format: org.apache.hadoop.mapred.TextInputFormat
-                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                  properties:
-                    SORTBUCKETCOLSPREFIX TRUE
-                    bucket_count 2
-                    bucket_field_name key
-                    bucketing_version 2
-                    column.name.delimiter ,
-                    columns key,val
-                    columns.types string:string
-#### A masked pattern was here ####
-                    name default.t1_n80
-                    serialization.format 1
-                    serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                
-                    input format: org.apache.hadoop.mapred.TextInputFormat
-                    output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                    properties:
-                      SORTBUCKETCOLSPREFIX TRUE
-                      bucket_count 2
-                      bucket_field_name key
-                      bucketing_version 2
-                      column.name.delimiter ,
-                      columns key,val
-                      columns.comments 
-                      columns.types string:string
-#### A masked pattern was here ####
-                      name default.t1_n80
-                      serialization.format 1
-                      serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                    name: default.t1_n80
-                  name: default.t1_n80
-            Truncated Path -> Alias:
-              /t1_n80 [t1_n80]
-        Map 3 
-            Map Operator Tree:
-                TableScan
-                  alias: t1_n80
-                  filterExpr: key is not null (type: boolean)
-                  Statistics: Num rows: 6 Data size: 2208 Basic stats: COMPLETE Column stats: NONE
-                  GatherStats: false
-                  Filter Operator
-                    isSamplingPred: false
-                    predicate: key is not null (type: boolean)
-                    Statistics: Num rows: 6 Data size: 2208 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
                       aggregations: count()
                       keys: key (type: string), val (type: string)
@@ -3371,7 +3316,7 @@ STAGE PLANS:
                   TotalFiles: 1
                   GatherStats: false
                   MultiFileSpray: false
-        Reducer 4 
+        Reducer 3 
             Execution mode: vectorized, llap
             Needs Tagging: false
             Reduce Operator Tree:
diff --git a/ql/src/test/results/clientpositive/llap/groupby_sort_skew_1_23.q.out b/ql/src/test/results/clientpositive/llap/groupby_sort_skew_1_23.q.out
index e0a465f..72a730e 100644
--- a/ql/src/test/results/clientpositive/llap/groupby_sort_skew_1_23.q.out
+++ b/ql/src/test/results/clientpositive/llap/groupby_sort_skew_1_23.q.out
@@ -3295,9 +3295,9 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
-        Reducer 4 <- Map 3 (SIMPLE_EDGE)
-        Reducer 5 <- Reducer 4 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE)
+        Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -3328,61 +3328,6 @@ STAGE PLANS:
                         tag: 0
                         value expressions: _col1 (type: bigint)
                         auto parallelism: true
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-            Path -> Alias:
-#### A masked pattern was here ####
-            Path -> Partition:
-#### A masked pattern was here ####
-                Partition
-                  base file name: t1_n56
-                  input format: org.apache.hadoop.mapred.TextInputFormat
-                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                  properties:
-                    SORTBUCKETCOLSPREFIX TRUE
-                    bucket_count 2
-                    bucket_field_name key
-                    bucketing_version 2
-                    column.name.delimiter ,
-                    columns key,val
-                    columns.types string:string
-#### A masked pattern was here ####
-                    name default.t1_n56
-                    serialization.format 1
-                    serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                
-                    input format: org.apache.hadoop.mapred.TextInputFormat
-                    output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                    properties:
-                      SORTBUCKETCOLSPREFIX TRUE
-                      bucket_count 2
-                      bucket_field_name key
-                      bucketing_version 2
-                      column.name.delimiter ,
-                      columns key,val
-                      columns.comments 
-                      columns.types string:string
-#### A masked pattern was here ####
-                      name default.t1_n56
-                      serialization.format 1
-                      serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                    name: default.t1_n56
-                  name: default.t1_n56
-            Truncated Path -> Alias:
-              /t1_n56 [t1_n56]
-        Map 3 
-            Map Operator Tree:
-                TableScan
-                  alias: t1_n56
-                  filterExpr: key is not null (type: boolean)
-                  Statistics: Num rows: 6 Data size: 2208 Basic stats: COMPLETE Column stats: NONE
-                  GatherStats: false
-                  Filter Operator
-                    isSamplingPred: false
-                    predicate: key is not null (type: boolean)
-                    Statistics: Num rows: 6 Data size: 2208 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
                       aggregations: count()
                       keys: key (type: string), val (type: string)
@@ -3482,7 +3427,7 @@ STAGE PLANS:
                   TotalFiles: 1
                   GatherStats: false
                   MultiFileSpray: false
-        Reducer 4 
+        Reducer 3 
             Execution mode: vectorized, llap
             Needs Tagging: false
             Reduce Operator Tree:
@@ -3503,7 +3448,7 @@ STAGE PLANS:
                   tag: -1
                   value expressions: _col2 (type: bigint)
                   auto parallelism: true
-        Reducer 5 
+        Reducer 4 
             Execution mode: vectorized, llap
             Needs Tagging: false
             Reduce Operator Tree:
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 0328e4b..af66068 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
@@ -976,17 +976,17 @@ STAGE PLANS:
 #### 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)
+        Map 7 <- Map 1 (BROADCAST_EDGE), Map 6 (BROADCAST_EDGE)
         Reducer 3 <- Map 2 (CUSTOM_SIMPLE_EDGE), Union 4 (CONTAINS)
         Reducer 5 <- Union 4 (SIMPLE_EDGE)
-        Reducer 9 <- Map 8 (CUSTOM_SIMPLE_EDGE), Union 4 (CONTAINS)
+        Reducer 8 <- Map 7 (CUSTOM_SIMPLE_EDGE), Union 4 (CONTAINS)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: x
-                  filterExpr: key is not null (type: boolean)
+                  filterExpr: (key is not null or value is not null) (type: boolean)
                   Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -997,23 +997,15 @@ STAGE PLANS:
                       sort order: +
                       Map-reduce partition columns: key (type: string)
                       Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 10 
-            Map Operator Tree:
-                TableScan
-                  alias: y
-                  filterExpr: value is not null (type: boolean)
-                  Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: value is not null (type: boolean)
-                    Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 25 Data size: 2225 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
                       key expressions: value (type: string)
                       null sort order: z
                       sort order: +
                       Map-reduce partition columns: value (type: string)
-                      Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 25 Data size: 2225 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 2 
@@ -1061,7 +1053,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: y
-                  filterExpr: key is not null (type: boolean)
+                  filterExpr: (key is not null or value is not null) (type: boolean)
                   Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -1072,26 +1064,18 @@ STAGE PLANS:
                       sort order: +
                       Map-reduce partition columns: key (type: string)
                       Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 7 
-            Map Operator Tree:
-                TableScan
-                  alias: x
-                  filterExpr: value is not null (type: boolean)
-                  Statistics: Num rows: 25 Data size: 2225 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: value is not null (type: boolean)
-                    Statistics: Num rows: 25 Data size: 2225 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
                       key expressions: value (type: string)
                       null sort order: z
                       sort order: +
                       Map-reduce partition columns: value (type: string)
-                      Statistics: Num rows: 25 Data size: 2225 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 8 
+        Map 7 
             Map Operator Tree:
                 TableScan
                   alias: z
@@ -1108,7 +1092,7 @@ STAGE PLANS:
                         1 value (type: string)
                       outputColumnNames: _col1
                       input vertices:
-                        0 Map 7
+                        0 Map 1
                       Statistics: Num rows: 162 Data size: 14418 Basic stats: COMPLETE Column stats: COMPLETE
                       Map Join Operator
                         condition map:
@@ -1117,7 +1101,7 @@ STAGE PLANS:
                           0 _col1 (type: string)
                           1 value (type: string)
                         input vertices:
-                          1 Map 10
+                          1 Map 6
                         Statistics: Num rows: 263 Data size: 2104 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
@@ -1167,7 +1151,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 9 
+        Reducer 8 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -1274,17 +1258,17 @@ STAGE PLANS:
 #### 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)
+        Map 7 <- Map 1 (BROADCAST_EDGE), Map 6 (BROADCAST_EDGE)
         Reducer 3 <- Map 2 (CUSTOM_SIMPLE_EDGE), Union 4 (CONTAINS)
         Reducer 5 <- Union 4 (SIMPLE_EDGE)
-        Reducer 9 <- Map 8 (CUSTOM_SIMPLE_EDGE), Union 4 (CONTAINS)
+        Reducer 8 <- Map 7 (CUSTOM_SIMPLE_EDGE), Union 4 (CONTAINS)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: x
-                  filterExpr: key is not null (type: boolean)
+                  filterExpr: (key is not null or value is not null) (type: boolean)
                   Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -1295,23 +1279,15 @@ STAGE PLANS:
                       sort order: +
                       Map-reduce partition columns: key (type: string)
                       Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 10 
-            Map Operator Tree:
-                TableScan
-                  alias: y
-                  filterExpr: value is not null (type: boolean)
-                  Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: value is not null (type: boolean)
-                    Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 25 Data size: 2225 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
                       key expressions: value (type: string)
                       null sort order: z
                       sort order: +
                       Map-reduce partition columns: value (type: string)
-                      Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 25 Data size: 2225 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 2 
@@ -1359,7 +1335,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: y
-                  filterExpr: key is not null (type: boolean)
+                  filterExpr: (key is not null or value is not null) (type: boolean)
                   Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -1370,26 +1346,18 @@ STAGE PLANS:
                       sort order: +
                       Map-reduce partition columns: key (type: string)
                       Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 7 
-            Map Operator Tree:
-                TableScan
-                  alias: x
-                  filterExpr: value is not null (type: boolean)
-                  Statistics: Num rows: 25 Data size: 2225 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: value is not null (type: boolean)
-                    Statistics: Num rows: 25 Data size: 2225 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
                       key expressions: value (type: string)
                       null sort order: z
                       sort order: +
                       Map-reduce partition columns: value (type: string)
-                      Statistics: Num rows: 25 Data size: 2225 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 8 
+        Map 7 
             Map Operator Tree:
                 TableScan
                   alias: z
@@ -1406,7 +1374,7 @@ STAGE PLANS:
                         1 value (type: string)
                       outputColumnNames: _col1
                       input vertices:
-                        0 Map 7
+                        0 Map 1
                       Statistics: Num rows: 162 Data size: 14418 Basic stats: COMPLETE Column stats: COMPLETE
                       Map Join Operator
                         condition map:
@@ -1415,7 +1383,7 @@ STAGE PLANS:
                           0 _col1 (type: string)
                           1 value (type: string)
                         input vertices:
-                          1 Map 10
+                          1 Map 6
                         Statistics: Num rows: 263 Data size: 2104 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: count()
@@ -1465,7 +1433,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 9 
+        Reducer 8 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
diff --git a/ql/src/test/results/clientpositive/llap/infer_join_preds.q.out b/ql/src/test/results/clientpositive/llap/infer_join_preds.q.out
index 2d8653c..a4977c2 100644
--- a/ql/src/test/results/clientpositive/llap/infer_join_preds.q.out
+++ b/ql/src/test/results/clientpositive/llap/infer_join_preds.q.out
@@ -1299,7 +1299,7 @@ STAGE PLANS:
 #### 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)
+        Reducer 3 <- Map 4 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -1343,17 +1343,6 @@ STAGE PLANS:
                         sort order: +
                         Map-reduce partition columns: _col0 (type: bigint)
                         Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 5 
-            Map Operator Tree:
-                TableScan
-                  alias: d
-                  filterExpr: prid is not null (type: boolean)
-                  Statistics: Num rows: 1 Data size: 776 Basic stats: COMPLETE Column stats: NONE
-                  Filter Operator
-                    predicate: prid is not null (type: boolean)
-                    Statistics: Num rows: 1 Data size: 776 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: prid (type: bigint), concat(CASE WHEN (length(pruid) is null) THEN ('') ELSE (pruid) END, ',', CASE WHEN (prid is null) THEN (1L) ELSE (prid) END, ',', CASE WHEN (prtimesheetid is null) THEN (1L) ELSE (prtimesheetid) END, ',', CASE WHEN (prassignmentid is null) THEN (1L) ELSE (prassignmentid) END, ',', CASE WHEN (prchargecodeid is null) THEN (1L) ELSE (prchargecodeid) END, ',', CASE WHEN (prtypecodeid is null) THEN ('') ELSE (CAST( prtypecodeid AS STRIN [...]
                       outputColumnNames: _col0, _col1
diff --git a/ql/src/test/results/clientpositive/llap/join12.q.out b/ql/src/test/results/clientpositive/llap/join12.q.out
index eb3f923..9d6dd35 100644
--- a/ql/src/test/results/clientpositive/llap/join12.q.out
+++ b/ql/src/test/results/clientpositive/llap/join12.q.out
@@ -34,7 +34,7 @@ STAGE PLANS:
 #### 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)
+        Reducer 3 <- Map 4 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -63,27 +63,6 @@ STAGE PLANS:
                 TableScan
                   alias: src
                   filterExpr: (UDFToDouble(key) < 80.0D) (type: boolean)
-                  Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: (UDFToDouble(key) < 80.0D) (type: boolean)
-                    Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE
-                    Select Operator
-                      expressions: key (type: string)
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: string)
-                        null sort order: z
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: string)
-                        Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 5 
-            Map Operator Tree:
-                TableScan
-                  alias: src
-                  filterExpr: (UDFToDouble(key) < 80.0D) (type: boolean)
                   Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (UDFToDouble(key) < 80.0D) (type: boolean)
@@ -99,6 +78,16 @@ STAGE PLANS:
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: string)
+                    Select Operator
+                      expressions: key (type: string)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
diff --git a/ql/src/test/results/clientpositive/llap/join2.q.out b/ql/src/test/results/clientpositive/llap/join2.q.out
index b1aaed7..598ff86 100644
--- a/ql/src/test/results/clientpositive/llap/join2.q.out
+++ b/ql/src/test/results/clientpositive/llap/join2.q.out
@@ -30,14 +30,14 @@ STAGE PLANS:
 #### 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)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: src1
-                  filterExpr: (key is not null and UDFToDouble(key) is not null) (type: boolean)
+                  filterExpr: ((key is not null and UDFToDouble(key) is not null) or UDFToDouble(key) is not null) (type: boolean)
                   Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key is not null and UDFToDouble(key) is not null) (type: boolean)
@@ -53,6 +53,20 @@ STAGE PLANS:
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: double)
+                  Filter Operator
+                    predicate: UDFToDouble(key) is not null (type: boolean)
+                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: value (type: string), UDFToDouble(key) (type: double)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 500 Data size: 49500 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col1 (type: double)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col1 (type: double)
+                        Statistics: Num rows: 500 Data size: 49500 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 4 
@@ -77,28 +91,6 @@ STAGE PLANS:
                         value expressions: _col1 (type: double)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 5 
-            Map Operator Tree:
-                TableScan
-                  alias: src3
-                  filterExpr: UDFToDouble(key) is not null (type: boolean)
-                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: UDFToDouble(key) is not null (type: boolean)
-                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                    Select Operator
-                      expressions: value (type: string), UDFToDouble(key) (type: double)
-                      outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 500 Data size: 49500 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col1 (type: double)
-                        null sort order: z
-                        sort order: +
-                        Map-reduce partition columns: _col1 (type: double)
-                        Statistics: Num rows: 500 Data size: 49500 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions: _col0 (type: string)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
diff --git a/ql/src/test/results/clientpositive/llap/join3.q.out b/ql/src/test/results/clientpositive/llap/join3.q.out
index 8de4227..4ffee5c 100644
--- a/ql/src/test/results/clientpositive/llap/join3.q.out
+++ b/ql/src/test/results/clientpositive/llap/join3.q.out
@@ -30,7 +30,7 @@ STAGE PLANS:
 #### 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)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
         Reducer 4 <- Reducer 3 (CUSTOM_SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
@@ -53,6 +53,17 @@ STAGE PLANS:
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: string), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 5 
@@ -76,28 +87,6 @@ STAGE PLANS:
                         Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 6 
-            Map Operator Tree:
-                TableScan
-                  alias: src3
-                  filterExpr: key is not null (type: boolean)
-                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: key is not null (type: boolean)
-                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                    Select Operator
-                      expressions: key (type: string), value (type: string)
-                      outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: string)
-                        null sort order: z
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: string)
-                        Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions: _col1 (type: string)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
diff --git a/ql/src/test/results/clientpositive/llap/join_alt_syntax.q.out b/ql/src/test/results/clientpositive/llap/join_alt_syntax.q.out
index ad7b562..c41bf0d 100644
--- a/ql/src/test/results/clientpositive/llap/join_alt_syntax.q.out
+++ b/ql/src/test/results/clientpositive/llap/join_alt_syntax.q.out
@@ -458,7 +458,7 @@ STAGE PLANS:
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-        Reducer 4 <- Map 1 (SIMPLE_EDGE), Map 6 (SIMPLE_EDGE)
+        Reducer 4 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -494,7 +494,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: p4
-                  filterExpr: p_partkey is not null (type: boolean)
+                  filterExpr: (p_partkey is not null or p_name is not null) (type: boolean)
                   Statistics: Num rows: 26 Data size: 3250 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: p_partkey is not null (type: boolean)
@@ -510,14 +510,6 @@ STAGE PLANS:
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 26 Data size: 3250 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: string)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 6 
-            Map Operator Tree:
-                TableScan
-                  alias: p3
-                  filterExpr: p_name is not null (type: boolean)
-                  Statistics: Num rows: 26 Data size: 3146 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: p_name is not null (type: boolean)
                     Statistics: Num rows: 26 Data size: 3146 Basic stats: COMPLETE Column stats: COMPLETE
@@ -623,7 +615,7 @@ STAGE PLANS:
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-        Reducer 4 <- Map 1 (SIMPLE_EDGE), Map 6 (SIMPLE_EDGE)
+        Reducer 4 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -659,7 +651,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: p4
-                  filterExpr: p_partkey is not null (type: boolean)
+                  filterExpr: (p_partkey is not null or p_name is not null) (type: boolean)
                   Statistics: Num rows: 26 Data size: 3250 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: p_partkey is not null (type: boolean)
@@ -675,14 +667,6 @@ STAGE PLANS:
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 26 Data size: 3250 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: string)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 6 
-            Map Operator Tree:
-                TableScan
-                  alias: p3
-                  filterExpr: p_name is not null (type: boolean)
-                  Statistics: Num rows: 26 Data size: 3146 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: p_name is not null (type: boolean)
                     Statistics: Num rows: 26 Data size: 3146 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/join_merging.q.out b/ql/src/test/results/clientpositive/llap/join_merging.q.out
index f32a740..e8b00c1 100644
--- a/ql/src/test/results/clientpositive/llap/join_merging.q.out
+++ b/ql/src/test/results/clientpositive/llap/join_merging.q.out
@@ -21,8 +21,8 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-        Reducer 4 <- Map 3 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -40,14 +40,6 @@ STAGE PLANS:
                       sort order: +
                       Map-reduce partition columns: _col0 (type: int)
                       Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 3 
-            Map Operator Tree:
-                TableScan
-                  alias: p2
-                  filterExpr: p_partkey is not null (type: boolean)
-                  Statistics: Num rows: 26 Data size: 208 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: p_partkey is not null (type: boolean)
                     Statistics: Num rows: 26 Data size: 208 Basic stats: COMPLETE Column stats: COMPLETE
@@ -64,7 +56,7 @@ STAGE PLANS:
                         value expressions: _col1 (type: int)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 5 
+        Map 4 
             Map Operator Tree:
                 TableScan
                   alias: p1
@@ -108,7 +100,7 @@ STAGE PLANS:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                         serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 4 
+        Reducer 3 
             Execution mode: llap
             Reduce Operator Tree:
               Merge Join Operator
@@ -160,8 +152,8 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-        Reducer 4 <- Map 3 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -179,14 +171,6 @@ STAGE PLANS:
                       sort order: +
                       Map-reduce partition columns: _col0 (type: int)
                       Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 3 
-            Map Operator Tree:
-                TableScan
-                  alias: p2
-                  filterExpr: (p_partkey is not null and p_size is not null) (type: boolean)
-                  Statistics: Num rows: 26 Data size: 208 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (p_partkey is not null and p_size is not null) (type: boolean)
                     Statistics: Num rows: 26 Data size: 208 Basic stats: COMPLETE Column stats: COMPLETE
@@ -203,7 +187,7 @@ STAGE PLANS:
                         value expressions: _col1 (type: int), _col2 (type: int)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 5 
+        Map 4 
             Map Operator Tree:
                 TableScan
                   alias: p1
@@ -247,7 +231,7 @@ STAGE PLANS:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                         serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 4 
+        Reducer 3 
             Execution mode: llap
             Reduce Operator Tree:
               Merge Join Operator
diff --git a/ql/src/test/results/clientpositive/llap/join_parse.q.out b/ql/src/test/results/clientpositive/llap/join_parse.q.out
index 8f8ea12..cc734fe 100644
--- a/ql/src/test/results/clientpositive/llap/join_parse.q.out
+++ b/ql/src/test/results/clientpositive/llap/join_parse.q.out
@@ -32,7 +32,7 @@ STAGE PLANS:
 #### 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)
+        Reducer 3 <- Map 4 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -61,7 +61,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: src
-                  filterExpr: key is not null (type: boolean)
+                  filterExpr: (key is not null or value is not null) (type: boolean)
                   Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -76,14 +76,6 @@ STAGE PLANS:
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 5 
-            Map Operator Tree:
-                TableScan
-                  alias: src1
-                  filterExpr: value is not null (type: boolean)
-                  Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: value is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
@@ -180,7 +172,7 @@ STAGE PLANS:
 #### 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)
+        Reducer 3 <- Map 4 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -209,7 +201,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: src
-                  filterExpr: key is not null (type: boolean)
+                  filterExpr: (key is not null or value is not null) (type: boolean)
                   Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -224,14 +216,6 @@ STAGE PLANS:
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 5 
-            Map Operator Tree:
-                TableScan
-                  alias: src1
-                  filterExpr: value is not null (type: boolean)
-                  Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: value is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
@@ -328,7 +312,7 @@ STAGE PLANS:
 #### 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)
+        Reducer 3 <- Map 4 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -357,7 +341,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: src
-                  filterExpr: key is not null (type: boolean)
+                  filterExpr: (key is not null or value is not null) (type: boolean)
                   Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -372,14 +356,6 @@ STAGE PLANS:
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 5 
-            Map Operator Tree:
-                TableScan
-                  alias: src1
-                  filterExpr: value is not null (type: boolean)
-                  Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: value is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/keep_uniform.q.out b/ql/src/test/results/clientpositive/llap/keep_uniform.q.out
index fba97d5..dac0a5a 100644
--- a/ql/src/test/results/clientpositive/llap/keep_uniform.q.out
+++ b/ql/src/test/results/clientpositive/llap/keep_uniform.q.out
@@ -437,25 +437,25 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 12 <- Map 11 (SIMPLE_EDGE), Map 17 (SIMPLE_EDGE)
+        Reducer 10 <- Reducer 9 (SIMPLE_EDGE)
+        Reducer 11 <- Map 1 (SIMPLE_EDGE), Map 17 (SIMPLE_EDGE)
+        Reducer 12 <- Map 1 (SIMPLE_EDGE), Reducer 11 (SIMPLE_EDGE)
         Reducer 13 <- Reducer 12 (SIMPLE_EDGE)
-        Reducer 14 <- Map 11 (SIMPLE_EDGE), Map 18 (SIMPLE_EDGE)
-        Reducer 15 <- Map 11 (SIMPLE_EDGE), Reducer 14 (SIMPLE_EDGE)
-        Reducer 16 <- Reducer 15 (SIMPLE_EDGE)
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 9 (SIMPLE_EDGE)
-        Reducer 3 <- Map 10 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
-        Reducer 4 <- Reducer 13 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
-        Reducer 5 <- Reducer 16 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-        Reducer 6 <- Map 19 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 14 (SIMPLE_EDGE)
+        Reducer 3 <- Map 15 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+        Reducer 4 <- Reducer 10 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+        Reducer 5 <- Reducer 13 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
+        Reducer 6 <- Map 18 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
         Reducer 7 <- Reducer 6 (SIMPLE_EDGE)
         Reducer 8 <- Reducer 7 (CUSTOM_SIMPLE_EDGE)
+        Reducer 9 <- Map 1 (SIMPLE_EDGE), Map 16 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: ws1
-                  filterExpr: (ws_order_number is not null and ws_ship_date_sk is not null and ws_ship_addr_sk is not null and ws_web_site_sk is not null) (type: boolean)
+                  filterExpr: ((ws_order_number is not null and ws_ship_date_sk is not null and ws_ship_addr_sk is not null and ws_web_site_sk is not null) or ws_order_number is not null) (type: boolean)
                   Statistics: Num rows: 1 Data size: 240 Basic stats: COMPLETE Column stats: NONE
                   TableScan Vectorization:
                       native: true
@@ -488,6 +488,60 @@ STAGE PLANS:
                             valueColumns: 2:int, 13:int, 17:int, 28:decimal(7,2), 33:decimal(7,2)
                         Statistics: Num rows: 1 Data size: 240 Basic stats: COMPLETE Column stats: NONE
                         value expressions: _col0 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: decimal(7,2)), _col5 (type: decimal(7,2))
+                  Filter Operator
+                    Filter Vectorization:
+                        className: VectorFilterOperator
+                        native: true
+                        predicateExpression: SelectColumnIsNotNull(col 17:int)
+                    predicate: ws_order_number is not null (type: boolean)
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: ws_warehouse_sk (type: int), ws_order_number (type: int)
+                      outputColumnNames: _col0, _col1
+                      Select Vectorization:
+                          className: VectorSelectOperator
+                          native: true
+                          projectedOutputColumnNums: [15, 17]
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col1 (type: int)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col1 (type: int)
+                        Reduce Sink Vectorization:
+                            className: VectorReduceSinkLongOperator
+                            keyColumns: 17:int
+                            native: true
+                            nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
+                            valueColumns: 15:int
+                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col0 (type: int)
+                      Reduce Output Operator
+                        key expressions: _col1 (type: int)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col1 (type: int)
+                        Reduce Sink Vectorization:
+                            className: VectorReduceSinkLongOperator
+                            keyColumns: 17:int
+                            native: true
+                            nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
+                            valueColumns: 15:int
+                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col0 (type: int)
+                      Reduce Output Operator
+                        key expressions: _col1 (type: int)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col1 (type: int)
+                        Reduce Sink Vectorization:
+                            className: VectorReduceSinkLongOperator
+                            keyColumns: 17:int
+                            native: true
+                            nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
+                            valueColumns: 15:int
+                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col0 (type: int)
             Execution mode: vectorized, llap
             LLAP IO: may be used (ACID table)
             Map Vectorization:
@@ -501,28 +555,28 @@ STAGE PLANS:
                 vectorized: true
                 rowBatchContext:
                     dataColumnCount: 34
-                    includeColumns: [2, 11, 13, 17, 28, 33]
+                    includeColumns: [2, 11, 13, 15, 17, 28, 33]
                     dataColumns: ws_sold_date_sk:int, ws_sold_time_sk:int, ws_ship_date_sk:int, ws_item_sk:int, ws_bill_customer_sk:int, ws_bill_cdemo_sk:int, ws_bill_hdemo_sk:int, ws_bill_addr_sk:int, ws_ship_customer_sk:int, ws_ship_cdemo_sk:int, ws_ship_hdemo_sk:int, ws_ship_addr_sk:int, ws_web_page_sk:int, ws_web_site_sk:int, ws_ship_mode_sk:int, ws_warehouse_sk:int, ws_promo_sk:int, ws_order_number:int, ws_quantity:int, ws_wholesale_cost:decimal(7,2)/DECIMAL_64, ws_list_price:decima [...]
                     partitionColumnCount: 0
                     scratchColumnTypeNames: []
-        Map 10 
+        Map 14 
             Map Operator Tree:
                 TableScan
-                  alias: web_site
-                  filterExpr: ((web_company_name = 'pri') and web_site_sk is not null) (type: boolean)
+                  alias: customer_address
+                  filterExpr: ((ca_state = 'TX') and ca_address_sk is not null) (type: boolean)
                   Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
                   TableScan Vectorization:
                       native: true
-                      vectorizationSchemaColumns: [0:web_site_sk:int, 1:web_site_id:string, 2:web_rec_start_date:string, 3:web_rec_end_date:string, 4:web_name:string, 5:web_open_date_sk:int, 6:web_close_date_sk:int, 7:web_class:string, 8:web_manager:string, 9:web_mkt_id:int, 10:web_mkt_class:string, 11:web_mkt_desc:string, 12:web_market_manager:string, 13:web_company_id:int, 14:web_company_name:string, 15:web_street_number:string, 16:web_street_name:string, 17:web_street_type:string, 18: [...]
+                      vectorizationSchemaColumns: [0:ca_address_sk:int, 1:ca_address_id:string, 2:ca_street_number:string, 3:ca_street_name:string, 4:ca_street_type:string, 5:ca_suite_number:string, 6:ca_city:string, 7:ca_county:string, 8:ca_state:string, 9:ca_zip:string, 10:ca_country:string, 11:ca_gmt_offset:decimal(5,2)/DECIMAL_64, 12:ca_location_type:string, 13:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
                   Filter Operator
                     Filter Vectorization:
                         className: VectorFilterOperator
                         native: true
-                        predicateExpression: FilterExprAndExpr(children: FilterStringGroupColEqualStringScalar(col 14:string, val pri), SelectColumnIsNotNull(col 0:int))
-                    predicate: ((web_company_name = 'pri') and web_site_sk is not null) (type: boolean)
+                        predicateExpression: FilterExprAndExpr(children: FilterStringGroupColEqualStringScalar(col 8:string, val TX), SelectColumnIsNotNull(col 0:int))
+                    predicate: ((ca_state = 'TX') and ca_address_sk is not null) (type: boolean)
                     Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      expressions: web_site_sk (type: int)
+                      expressions: ca_address_sk (type: int)
                       outputColumnNames: _col0
                       Select Vectorization:
                           className: VectorSelectOperator
@@ -552,74 +606,46 @@ STAGE PLANS:
                 usesVectorUDFAdaptor: false
                 vectorized: true
                 rowBatchContext:
-                    dataColumnCount: 26
-                    includeColumns: [0, 14]
-                    dataColumns: web_site_sk:int, web_site_id:string, web_rec_start_date:string, web_rec_end_date:string, web_name:string, web_open_date_sk:int, web_close_date_sk:int, web_class:string, web_manager:string, web_mkt_id:int, web_mkt_class:string, web_mkt_desc:string, web_market_manager:string, web_company_id:int, web_company_name:string, web_street_number:string, web_street_name:string, web_street_type:string, web_suite_number:string, web_city:string, web_county:string, web_ [...]
+                    dataColumnCount: 13
+                    includeColumns: [0, 8]
+                    dataColumns: ca_address_sk:int, ca_address_id:string, ca_street_number:string, ca_street_name:string, ca_street_type:string, ca_suite_number:string, ca_city:string, ca_county:string, ca_state:string, ca_zip:string, ca_country:string, ca_gmt_offset:decimal(5,2)/DECIMAL_64, ca_location_type:string
                     partitionColumnCount: 0
                     scratchColumnTypeNames: []
-        Map 11 
+        Map 15 
             Map Operator Tree:
                 TableScan
-                  alias: ws1
-                  filterExpr: ws_order_number is not null (type: boolean)
-                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  alias: web_site
+                  filterExpr: ((web_company_name = 'pri') and web_site_sk is not null) (type: boolean)
+                  Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
                   TableScan Vectorization:
                       native: true
-                      vectorizationSchemaColumns: [0:ws_sold_date_sk:int, 1:ws_sold_time_sk:int, 2:ws_ship_date_sk:int, 3:ws_item_sk:int, 4:ws_bill_customer_sk:int, 5:ws_bill_cdemo_sk:int, 6:ws_bill_hdemo_sk:int, 7:ws_bill_addr_sk:int, 8:ws_ship_customer_sk:int, 9:ws_ship_cdemo_sk:int, 10:ws_ship_hdemo_sk:int, 11:ws_ship_addr_sk:int, 12:ws_web_page_sk:int, 13:ws_web_site_sk:int, 14:ws_ship_mode_sk:int, 15:ws_warehouse_sk:int, 16:ws_promo_sk:int, 17:ws_order_number:int, 18:ws_quantity:int [...]
+                      vectorizationSchemaColumns: [0:web_site_sk:int, 1:web_site_id:string, 2:web_rec_start_date:string, 3:web_rec_end_date:string, 4:web_name:string, 5:web_open_date_sk:int, 6:web_close_date_sk:int, 7:web_class:string, 8:web_manager:string, 9:web_mkt_id:int, 10:web_mkt_class:string, 11:web_mkt_desc:string, 12:web_market_manager:string, 13:web_company_id:int, 14:web_company_name:string, 15:web_street_number:string, 16:web_street_name:string, 17:web_street_type:string, 18: [...]
                   Filter Operator
                     Filter Vectorization:
                         className: VectorFilterOperator
                         native: true
-                        predicateExpression: SelectColumnIsNotNull(col 17:int)
-                    predicate: ws_order_number is not null (type: boolean)
-                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                        predicateExpression: FilterExprAndExpr(children: FilterStringGroupColEqualStringScalar(col 14:string, val pri), SelectColumnIsNotNull(col 0:int))
+                    predicate: ((web_company_name = 'pri') and web_site_sk is not null) (type: boolean)
+                    Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      expressions: ws_warehouse_sk (type: int), ws_order_number (type: int)
-                      outputColumnNames: _col0, _col1
+                      expressions: web_site_sk (type: int)
+                      outputColumnNames: _col0
                       Select Vectorization:
                           className: VectorSelectOperator
                           native: true
-                          projectedOutputColumnNums: [15, 17]
-                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                      Reduce Output Operator
-                        key expressions: _col1 (type: int)
-                        null sort order: z
-                        sort order: +
-                        Map-reduce partition columns: _col1 (type: int)
-                        Reduce Sink Vectorization:
-                            className: VectorReduceSinkLongOperator
-                            keyColumns: 17:int
-                            native: true
-                            nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
-                            valueColumns: 15:int
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                        value expressions: _col0 (type: int)
-                      Reduce Output Operator
-                        key expressions: _col1 (type: int)
-                        null sort order: z
-                        sort order: +
-                        Map-reduce partition columns: _col1 (type: int)
-                        Reduce Sink Vectorization:
-                            className: VectorReduceSinkLongOperator
-                            keyColumns: 17:int
-                            native: true
-                            nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
-                            valueColumns: 15:int
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                        value expressions: _col0 (type: int)
+                          projectedOutputColumnNums: [0]
+                      Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
-                        key expressions: _col1 (type: int)
+                        key expressions: _col0 (type: int)
                         null sort order: z
                         sort order: +
-                        Map-reduce partition columns: _col1 (type: int)
+                        Map-reduce partition columns: _col0 (type: int)
                         Reduce Sink Vectorization:
                             className: VectorReduceSinkLongOperator
-                            keyColumns: 17:int
+                            keyColumns: 0:int
                             native: true
                             nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
-                            valueColumns: 15:int
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                        value expressions: _col0 (type: int)
+                        Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
             Execution mode: vectorized, llap
             LLAP IO: may be used (ACID table)
             Map Vectorization:
@@ -632,12 +658,12 @@ STAGE PLANS:
                 usesVectorUDFAdaptor: false
                 vectorized: true
                 rowBatchContext:
-                    dataColumnCount: 34
-                    includeColumns: [15, 17]
-                    dataColumns: ws_sold_date_sk:int, ws_sold_time_sk:int, ws_ship_date_sk:int, ws_item_sk:int, ws_bill_customer_sk:int, ws_bill_cdemo_sk:int, ws_bill_hdemo_sk:int, ws_bill_addr_sk:int, ws_ship_customer_sk:int, ws_ship_cdemo_sk:int, ws_ship_hdemo_sk:int, ws_ship_addr_sk:int, ws_web_page_sk:int, ws_web_site_sk:int, ws_ship_mode_sk:int, ws_warehouse_sk:int, ws_promo_sk:int, ws_order_number:int, ws_quantity:int, ws_wholesale_cost:decimal(7,2)/DECIMAL_64, ws_list_price:decima [...]
+                    dataColumnCount: 26
+                    includeColumns: [0, 14]
+                    dataColumns: web_site_sk:int, web_site_id:string, web_rec_start_date:string, web_rec_end_date:string, web_name:string, web_open_date_sk:int, web_close_date_sk:int, web_class:string, web_manager:string, web_mkt_id:int, web_mkt_class:string, web_mkt_desc:string, web_market_manager:string, web_company_id:int, web_company_name:string, web_street_number:string, web_street_name:string, web_street_type:string, web_suite_number:string, web_city:string, web_county:string, web_ [...]
                     partitionColumnCount: 0
                     scratchColumnTypeNames: []
-        Map 17 
+        Map 16 
             Map Operator Tree:
                 TableScan
                   alias: ws2
@@ -691,7 +717,7 @@ STAGE PLANS:
                     dataColumns: ws_sold_date_sk:int, ws_sold_time_sk:int, ws_ship_date_sk:int, ws_item_sk:int, ws_bill_customer_sk:int, ws_bill_cdemo_sk:int, ws_bill_hdemo_sk:int, ws_bill_addr_sk:int, ws_ship_customer_sk:int, ws_ship_cdemo_sk:int, ws_ship_hdemo_sk:int, ws_ship_addr_sk:int, ws_web_page_sk:int, ws_web_site_sk:int, ws_ship_mode_sk:int, ws_warehouse_sk:int, ws_promo_sk:int, ws_order_number:int, ws_quantity:int, ws_wholesale_cost:decimal(7,2)/DECIMAL_64, ws_list_price:decima [...]
                     partitionColumnCount: 0
                     scratchColumnTypeNames: []
-        Map 18 
+        Map 17 
             Map Operator Tree:
                 TableScan
                   alias: web_returns
@@ -743,7 +769,7 @@ STAGE PLANS:
                     dataColumns: wr_returned_date_sk:int, wr_returned_time_sk:int, wr_item_sk:int, wr_refunded_customer_sk:int, wr_refunded_cdemo_sk:int, wr_refunded_hdemo_sk:int, wr_refunded_addr_sk:int, wr_returning_customer_sk:int, wr_returning_cdemo_sk:int, wr_returning_hdemo_sk:int, wr_returning_addr_sk:int, wr_web_page_sk:int, wr_reason_sk:int, wr_order_number:int, wr_return_quantity:int, wr_return_amt:decimal(7,2)/DECIMAL_64, wr_return_tax:decimal(7,2)/DECIMAL_64, wr_return_amt_in [...]
                     partitionColumnCount: 0
                     scratchColumnTypeNames: []
-        Map 19 
+        Map 18 
             Map Operator Tree:
                 TableScan
                   alias: date_dim
@@ -795,90 +821,7 @@ STAGE PLANS:
                     dataColumns: d_date_sk:int, d_date_id:string, d_date:string, d_month_seq:int, d_week_seq:int, d_quarter_seq:int, d_year:int, d_dow:int, d_moy:int, d_dom:int, d_qoy:int, d_fy_year:int, d_fy_quarter_seq:int, d_fy_week_seq:int, d_day_name:string, d_quarter_name:string, d_holiday:string, d_weekend:string, d_following_holiday:string, d_first_dom:int, d_last_dom:int, d_same_day_ly:int, d_same_day_lq:int, d_current_day:string, d_current_week:string, d_current_month:string, d [...]
                     partitionColumnCount: 0
                     scratchColumnTypeNames: [timestamp]
-        Map 9 
-            Map Operator Tree:
-                TableScan
-                  alias: customer_address
-                  filterExpr: ((ca_state = 'TX') and ca_address_sk is not null) (type: boolean)
-                  Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                  TableScan Vectorization:
-                      native: true
-                      vectorizationSchemaColumns: [0:ca_address_sk:int, 1:ca_address_id:string, 2:ca_street_number:string, 3:ca_street_name:string, 4:ca_street_type:string, 5:ca_suite_number:string, 6:ca_city:string, 7:ca_county:string, 8:ca_state:string, 9:ca_zip:string, 10:ca_country:string, 11:ca_gmt_offset:decimal(5,2)/DECIMAL_64, 12:ca_location_type:string, 13:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
-                  Filter Operator
-                    Filter Vectorization:
-                        className: VectorFilterOperator
-                        native: true
-                        predicateExpression: FilterExprAndExpr(children: FilterStringGroupColEqualStringScalar(col 8:string, val TX), SelectColumnIsNotNull(col 0:int))
-                    predicate: ((ca_state = 'TX') and ca_address_sk is not null) (type: boolean)
-                    Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: ca_address_sk (type: int)
-                      outputColumnNames: _col0
-                      Select Vectorization:
-                          className: VectorSelectOperator
-                          native: true
-                          projectedOutputColumnNums: [0]
-                      Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: int)
-                        null sort order: z
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
-                        Reduce Sink Vectorization:
-                            className: VectorReduceSinkLongOperator
-                            keyColumns: 0:int
-                            native: true
-                            nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
-                        Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-            Execution mode: vectorized, llap
-            LLAP IO: may be used (ACID table)
-            Map Vectorization:
-                enabled: true
-                enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true
-                inputFormatFeatureSupport: [DECIMAL_64]
-                featureSupportInUse: [DECIMAL_64]
-                inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
-                allNative: true
-                usesVectorUDFAdaptor: false
-                vectorized: true
-                rowBatchContext:
-                    dataColumnCount: 13
-                    includeColumns: [0, 8]
-                    dataColumns: ca_address_sk:int, ca_address_id:string, ca_street_number:string, ca_street_name:string, ca_street_type:string, ca_suite_number:string, ca_city:string, ca_county:string, ca_state:string, ca_zip:string, ca_country:string, ca_gmt_offset:decimal(5,2)/DECIMAL_64, ca_location_type:string
-                    partitionColumnCount: 0
-                    scratchColumnTypeNames: []
-        Reducer 12 
-            Execution mode: llap
-            Reduce Operator Tree:
-              Merge Join Operator
-                condition map:
-                     Inner Join 0 to 1
-                keys:
-                  0 _col1 (type: int)
-                  1 _col1 (type: int)
-                outputColumnNames: _col0, _col1, _col2
-                residual filter predicates: {(_col0 <> _col2)}
-                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                Select Operator
-                  expressions: _col1 (type: int)
-                  outputColumnNames: _col1
-                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                  Group By Operator
-                    keys: _col1 (type: int)
-                    minReductionHashAggr: 0.99
-                    mode: hash
-                    outputColumnNames: _col0
-                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      null sort order: z
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
-                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-            MergeJoin Vectorization:
-                enabled: false
-                enableConditionsNotMet: Vectorizing MergeJoin Supported IS false
-        Reducer 13 
+        Reducer 10 
             Execution mode: vectorized, llap
             Reduce Vectorization:
                 enabled: true
@@ -917,7 +860,7 @@ STAGE PLANS:
                       native: true
                       nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-        Reducer 14 
+        Reducer 11 
             Execution mode: llap
             Reduce Operator Tree:
               Merge Join Operator
@@ -938,7 +881,7 @@ STAGE PLANS:
             MergeJoin Vectorization:
                 enabled: false
                 enableConditionsNotMet: Vectorizing MergeJoin Supported IS false
-        Reducer 15 
+        Reducer 12 
             Execution mode: llap
             Reduce Operator Tree:
               Merge Join Operator
@@ -969,7 +912,7 @@ STAGE PLANS:
             MergeJoin Vectorization:
                 enabled: false
                 enableConditionsNotMet: Vectorizing MergeJoin Supported IS false
-        Reducer 16 
+        Reducer 13 
             Execution mode: vectorized, llap
             Reduce Vectorization:
                 enabled: true
@@ -1210,6 +1153,37 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Reducer 9 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col1 (type: int)
+                  1 _col1 (type: int)
+                outputColumnNames: _col0, _col1, _col2
+                residual filter predicates: {(_col0 <> _col2)}
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                Select Operator
+                  expressions: _col1 (type: int)
+                  outputColumnNames: _col1
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  Group By Operator
+                    keys: _col1 (type: int)
+                    minReductionHashAggr: 0.99
+                    mode: hash
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: _col0 (type: int)
+                      null sort order: z
+                      sort order: +
+                      Map-reduce partition columns: _col0 (type: int)
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+            MergeJoin Vectorization:
+                enabled: false
+                enableConditionsNotMet: Vectorizing MergeJoin Supported IS false
 
   Stage: Stage-0
     Fetch Operator
diff --git a/ql/src/test/results/clientpositive/llap/leftsemijoin.q.out b/ql/src/test/results/clientpositive/llap/leftsemijoin.q.out
index fabc698..f72ff6f 100644
--- a/ql/src/test/results/clientpositive/llap/leftsemijoin.q.out
+++ b/ql/src/test/results/clientpositive/llap/leftsemijoin.q.out
@@ -108,7 +108,7 @@ POSTHOOK: query: drop table things_n1
 POSTHOOK: type: DROPTABLE
 POSTHOOK: Input: default@things_n1
 POSTHOOK: Output: default@things_n1
-Warning: Shuffle Join MERGEJOIN[34][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product
+Warning: Shuffle Join MERGEJOIN[34][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: explain select part.p_type from part join (select p1.p_name from part p1, part p2 group by p1.p_name) pp ON pp.p_name = part.p_name
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
@@ -126,8 +126,8 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-        Reducer 4 <- Map 3 (XPROD_EDGE), Map 5 (XPROD_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (XPROD_EDGE), Map 4 (XPROD_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -150,17 +150,6 @@ STAGE PLANS:
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 26 Data size: 5850 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: string)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 3 
-            Map Operator Tree:
-                TableScan
-                  alias: p1
-                  filterExpr: p_name is not null (type: boolean)
-                  Statistics: Num rows: 26 Data size: 3146 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: p_name is not null (type: boolean)
-                    Statistics: Num rows: 26 Data size: 3146 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: p_name (type: string)
                       outputColumnNames: _col0
@@ -172,7 +161,7 @@ STAGE PLANS:
                         value expressions: _col0 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 5 
+        Map 4 
             Map Operator Tree:
                 TableScan
                   alias: p2
@@ -207,7 +196,7 @@ STAGE PLANS:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                         serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 4 
+        Reducer 3 
             Execution mode: llap
             Reduce Operator Tree:
               Merge Join Operator
@@ -237,7 +226,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[34][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product
+Warning: Shuffle Join MERGEJOIN[34][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: select part.p_type from part join (select p1.p_name from part p1, part p2 group by p1.p_name) pp ON pp.p_name = part.p_name
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
diff --git a/ql/src/test/results/clientpositive/llap/limit_join_transpose.q.out b/ql/src/test/results/clientpositive/llap/limit_join_transpose.q.out
index 504556f..364fe4b 100644
--- a/ql/src/test/results/clientpositive/llap/limit_join_transpose.q.out
+++ b/ql/src/test/results/clientpositive/llap/limit_join_transpose.q.out
@@ -739,7 +739,7 @@ STAGE PLANS:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
         Reducer 4 <- Map 1 (SIMPLE_EDGE)
-        Reducer 5 <- Map 7 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
+        Reducer 5 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
         Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
@@ -778,14 +778,6 @@ STAGE PLANS:
                         sort order: +
                         Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: string)
-            Execution mode: llap
-            LLAP IO: no inputs
-        Map 7 
-            Map Operator Tree:
-                TableScan
-                  alias: src3
-                  filterExpr: value is not null (type: boolean)
-                  Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: value is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1735,7 +1727,7 @@ STAGE PLANS:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
         Reducer 4 <- Map 1 (SIMPLE_EDGE)
-        Reducer 5 <- Map 7 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
+        Reducer 5 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
         Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
@@ -1774,14 +1766,6 @@ STAGE PLANS:
                         sort order: +
                         Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: string)
-            Execution mode: llap
-            LLAP IO: no inputs
-        Map 7 
-            Map Operator Tree:
-                TableScan
-                  alias: src3
-                  filterExpr: value is not null (type: boolean)
-                  Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: value is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/mapjoin_hint.q.out b/ql/src/test/results/clientpositive/llap/mapjoin_hint.q.out
index 3c044b4..57c9abc 100644
--- a/ql/src/test/results/clientpositive/llap/mapjoin_hint.q.out
+++ b/ql/src/test/results/clientpositive/llap/mapjoin_hint.q.out
@@ -478,7 +478,7 @@ STAGE PLANS:
 #### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 5 (XPROD_EDGE)
-        Reducer 3 <- Map 6 (XPROD_EDGE), Reducer 2 (XPROD_EDGE)
+        Reducer 3 <- Map 4 (XPROD_EDGE), Reducer 2 (XPROD_EDGE)
         Reducer 5 <- Map 4 (CUSTOM_SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
@@ -525,17 +525,6 @@ STAGE PLANS:
                           sort order: 
                           Statistics: Num rows: 1 Data size: 196 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col0 (type: bigint)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 6 
-            Map Operator Tree:
-                TableScan
-                  alias: part_null_n1
-                  filterExpr: p_name is null (type: boolean)
-                  Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE
-                  Filter Operator
-                    predicate: p_name is null (type: boolean)
-                    Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE
                       Limit
@@ -626,7 +615,7 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Map 1 <- Map 4 (BROADCAST_EDGE), Reducer 3 (BROADCAST_EDGE)
+        Map 1 <- Map 2 (BROADCAST_EDGE), Reducer 3 (BROADCAST_EDGE)
         Reducer 3 <- Map 2 (CUSTOM_SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
@@ -660,7 +649,7 @@ STAGE PLANS:
                             1 
                           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                           input vertices:
-                            1 Map 4
+                            1 Map 2
                           Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
                           File Output Operator
                             compressed: false
@@ -695,17 +684,6 @@ STAGE PLANS:
                           sort order: 
                           Statistics: Num rows: 1 Data size: 196 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col0 (type: bigint)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 4 
-            Map Operator Tree:
-                TableScan
-                  alias: part_null_n1
-                  filterExpr: p_name is null (type: boolean)
-                  Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE
-                  Filter Operator
-                    predicate: p_name is null (type: boolean)
-                    Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE
                       Limit
diff --git a/ql/src/test/results/clientpositive/llap/masking_12.q.out b/ql/src/test/results/clientpositive/llap/masking_12.q.out
index 040e52c..f8e6f5f 100644
--- a/ql/src/test/results/clientpositive/llap/masking_12.q.out
+++ b/ql/src/test/results/clientpositive/llap/masking_12.q.out
@@ -120,8 +120,8 @@ STAGE PLANS:
 #### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-        Reducer 4 <- Map 3 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
-        Reducer 6 <- Map 5 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE)
+        Reducer 4 <- Map 5 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -146,9 +146,21 @@ STAGE PLANS:
                         Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col0 (type: int)
                         Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: string)
+                    Group By Operator
+                      keys: key (type: int)
+                      minReductionHashAggr: 0.5
+                      mode: hash
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 250 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 250 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 3 
+        Map 5 
             Map Operator Tree:
                 TableScan
                   alias: src
@@ -172,31 +184,6 @@ STAGE PLANS:
                         value expressions: _col0 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 5 
-            Map Operator Tree:
-                TableScan
-                  alias: masking_test_subq_n1
-                  filterExpr: (UDFToDouble(key) is not null and key is not null) (type: boolean)
-                  properties:
-                    insideView TRUE
-                  Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: (UDFToDouble(key) is not null and key is not null) (type: boolean)
-                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                    Group By Operator
-                      keys: key (type: int)
-                      minReductionHashAggr: 0.5
-                      mode: hash
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 250 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: int)
-                        null sort order: z
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 250 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
@@ -218,6 +205,25 @@ STAGE PLANS:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                         serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Reducer 3 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                keys: KEY._col0 (type: int)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 250 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
+                Select Operator
+                  expressions: _col0 (type: int), UDFToDouble(_col0) (type: double)
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 250 Data size: 3000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Reduce Output Operator
+                    key expressions: _col1 (type: double)
+                    null sort order: z
+                    sort order: +
+                    Map-reduce partition columns: _col1 (type: double)
+                    Statistics: Num rows: 250 Data size: 3000 Basic stats: COMPLETE Column stats: COMPLETE
+                    value expressions: _col0 (type: int)
         Reducer 4 
             Execution mode: llap
             Reduce Operator Tree:
@@ -245,25 +251,6 @@ STAGE PLANS:
                       sort order: ++
                       Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col1 (type: int)
                       Statistics: Num rows: 197 Data size: 17927 Basic stats: COMPLETE Column stats: COMPLETE
-        Reducer 6 
-            Execution mode: vectorized, llap
-            Reduce Operator Tree:
-              Group By Operator
-                keys: KEY._col0 (type: int)
-                mode: mergepartial
-                outputColumnNames: _col0
-                Statistics: Num rows: 250 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: _col0 (type: int), UDFToDouble(_col0) (type: double)
-                  outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 250 Data size: 3000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    key expressions: _col1 (type: double)
-                    null sort order: z
-                    sort order: +
-                    Map-reduce partition columns: _col1 (type: double)
-                    Statistics: Num rows: 250 Data size: 3000 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col0 (type: int)
 
   Stage: Stage-0
     Fetch Operator
diff --git a/ql/src/test/results/clientpositive/llap/masking_3.q.out b/ql/src/test/results/clientpositive/llap/masking_3.q.out
index a359d3c..75eaeb1 100644
--- a/ql/src/test/results/clientpositive/llap/masking_3.q.out
+++ b/ql/src/test/results/clientpositive/llap/masking_3.q.out
@@ -30,8 +30,8 @@ STAGE PLANS:
 #### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-        Reducer 4 <- Map 3 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
-        Reducer 6 <- Map 5 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE)
+        Reducer 4 <- Map 5 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -54,9 +54,21 @@ STAGE PLANS:
                         Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col0 (type: int)
                         Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: string)
+                    Group By Operator
+                      keys: key (type: int)
+                      minReductionHashAggr: 0.5
+                      mode: hash
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 250 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 250 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 3 
+        Map 5 
             Map Operator Tree:
                 TableScan
                   alias: src
@@ -78,29 +90,6 @@ STAGE PLANS:
                         value expressions: _col0 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 5 
-            Map Operator Tree:
-                TableScan
-                  alias: masking_test_subq_n3
-                  filterExpr: (UDFToDouble(key) is not null and key is not null) (type: boolean)
-                  Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: (UDFToDouble(key) is not null and key is not null) (type: boolean)
-                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                    Group By Operator
-                      keys: key (type: int)
-                      minReductionHashAggr: 0.5
-                      mode: hash
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 250 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: int)
-                        null sort order: z
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 250 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
@@ -119,6 +108,25 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Reducer 3 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                keys: KEY._col0 (type: int)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 250 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
+                Select Operator
+                  expressions: _col0 (type: int), UDFToDouble(_col0) (type: double)
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 250 Data size: 3000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Reduce Output Operator
+                    key expressions: _col1 (type: double)
+                    null sort order: z
+                    sort order: +
+                    Map-reduce partition columns: _col1 (type: double)
+                    Statistics: Num rows: 250 Data size: 3000 Basic stats: COMPLETE Column stats: COMPLETE
+                    value expressions: _col0 (type: int)
         Reducer 4 
             Execution mode: llap
             Reduce Operator Tree:
@@ -146,25 +154,6 @@ STAGE PLANS:
                       sort order: ++
                       Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col1 (type: int)
                       Statistics: Num rows: 197 Data size: 17927 Basic stats: COMPLETE Column stats: COMPLETE
-        Reducer 6 
-            Execution mode: vectorized, llap
-            Reduce Operator Tree:
-              Group By Operator
-                keys: KEY._col0 (type: int)
-                mode: mergepartial
-                outputColumnNames: _col0
-                Statistics: Num rows: 250 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: _col0 (type: int), UDFToDouble(_col0) (type: double)
-                  outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 250 Data size: 3000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    key expressions: _col1 (type: double)
-                    null sort order: z
-                    sort order: +
-                    Map-reduce partition columns: _col1 (type: double)
-                    Statistics: Num rows: 250 Data size: 3000 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col0 (type: int)
 
   Stage: Stage-0
     Fetch Operator
@@ -702,8 +691,8 @@ STAGE PLANS:
 #### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-        Reducer 4 <- Map 3 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
-        Reducer 6 <- Map 5 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE)
+        Reducer 4 <- Map 5 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -726,9 +715,21 @@ STAGE PLANS:
                         Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col0 (type: int)
                         Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: string)
+                    Group By Operator
+                      keys: key (type: int)
+                      minReductionHashAggr: 0.5
+                      mode: hash
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 250 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 250 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 3 
+        Map 5 
             Map Operator Tree:
                 TableScan
                   alias: src
@@ -750,29 +751,6 @@ STAGE PLANS:
                         value expressions: _col0 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 5 
-            Map Operator Tree:
-                TableScan
-                  alias: masking_test_subq_n3
-                  filterExpr: ((key > 0) and UDFToDouble(key) is not null) (type: boolean)
-                  Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: ((key > 0) and UDFToDouble(key) is not null) (type: boolean)
-                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                    Group By Operator
-                      keys: key (type: int)
-                      minReductionHashAggr: 0.5
-                      mode: hash
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 250 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: int)
-                        null sort order: z
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 250 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
@@ -791,6 +769,25 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Reducer 3 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                keys: KEY._col0 (type: int)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 250 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
+                Select Operator
+                  expressions: _col0 (type: int), UDFToDouble(_col0) (type: double)
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 250 Data size: 3000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Reduce Output Operator
+                    key expressions: _col1 (type: double)
+                    null sort order: z
+                    sort order: +
+                    Map-reduce partition columns: _col1 (type: double)
+                    Statistics: Num rows: 250 Data size: 3000 Basic stats: COMPLETE Column stats: COMPLETE
+                    value expressions: _col0 (type: int)
         Reducer 4 
             Execution mode: llap
             Reduce Operator Tree:
@@ -818,25 +815,6 @@ STAGE PLANS:
                       sort order: ++
                       Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col1 (type: int)
                       Statistics: Num rows: 197 Data size: 17927 Basic stats: COMPLETE Column stats: COMPLETE
-        Reducer 6 
-            Execution mode: vectorized, llap
-            Reduce Operator Tree:
-              Group By Operator
-                keys: KEY._col0 (type: int)
-                mode: mergepartial
-                outputColumnNames: _col0
-                Statistics: Num rows: 250 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: _col0 (type: int), UDFToDouble(_col0) (type: double)
-                  outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 250 Data size: 3000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    key expressions: _col1 (type: double)
-                    null sort order: z
-                    sort order: +
-                    Map-reduce partition columns: _col1 (type: double)
-                    Statistics: Num rows: 250 Data size: 3000 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col0 (type: int)
 
   Stage: Stage-0
     Fetch Operator
@@ -2028,8 +2006,8 @@ STAGE PLANS:
 #### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-        Reducer 4 <- Map 3 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
-        Reducer 6 <- Map 5 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE)
+        Reducer 4 <- Map 5 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -2052,9 +2030,21 @@ STAGE PLANS:
                         Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col0 (type: int)
                         Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: string)
+                    Group By Operator
+                      keys: key (type: int)
+                      minReductionHashAggr: 0.5
+                      mode: hash
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 250 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 250 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 3 
+        Map 5 
             Map Operator Tree:
                 TableScan
                   alias: src
@@ -2076,29 +2066,6 @@ STAGE PLANS:
                         value expressions: _col0 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 5 
-            Map Operator Tree:
-                TableScan
-                  alias: masking_test_subq_n3
-                  filterExpr: ((key > 0) and UDFToDouble(key) is not null) (type: boolean)
-                  Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: ((key > 0) and UDFToDouble(key) is not null) (type: boolean)
-                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                    Group By Operator
-                      keys: key (type: int)
-                      minReductionHashAggr: 0.5
-                      mode: hash
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 250 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: int)
-                        null sort order: z
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 250 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
@@ -2121,6 +2088,25 @@ STAGE PLANS:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                         serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Reducer 3 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                keys: KEY._col0 (type: int)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 250 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
+                Select Operator
+                  expressions: _col0 (type: int), UDFToDouble(_col0) (type: double)
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 250 Data size: 3000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Reduce Output Operator
+                    key expressions: _col1 (type: double)
+                    null sort order: z
+                    sort order: +
+                    Map-reduce partition columns: _col1 (type: double)
+                    Statistics: Num rows: 250 Data size: 3000 Basic stats: COMPLETE Column stats: COMPLETE
+                    value expressions: _col0 (type: int)
         Reducer 4 
             Execution mode: llap
             Reduce Operator Tree:
@@ -2148,25 +2134,6 @@ STAGE PLANS:
                       sort order: ++
                       Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col1 (type: int)
                       Statistics: Num rows: 197 Data size: 17927 Basic stats: COMPLETE Column stats: COMPLETE
-        Reducer 6 
-            Execution mode: vectorized, llap
-            Reduce Operator Tree:
-              Group By Operator
-                keys: KEY._col0 (type: int)
-                mode: mergepartial
-                outputColumnNames: _col0
-                Statistics: Num rows: 250 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: _col0 (type: int), UDFToDouble(_col0) (type: double)
-                  outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 250 Data size: 3000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    key expressions: _col1 (type: double)
-                    null sort order: z
-                    sort order: +
-                    Map-reduce partition columns: _col1 (type: double)
-                    Statistics: Num rows: 250 Data size: 3000 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col0 (type: int)
 
   Stage: Stage-0
     Fetch Operator
@@ -2712,8 +2679,8 @@ STAGE PLANS:
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
         Reducer 4 <- Map 3 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
-        Reducer 6 <- Map 5 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE)
-        Reducer 8 <- Map 7 (SIMPLE_EDGE)
+        Reducer 5 <- Map 3 (SIMPLE_EDGE)
+        Reducer 6 <- Map 7 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -2758,9 +2725,21 @@ STAGE PLANS:
                         Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col0 (type: int)
                         Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: string)
+                    Group By Operator
+                      keys: key (type: int)
+                      minReductionHashAggr: 0.5
+                      mode: hash
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 250 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 250 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 5 
+        Map 7 
             Map Operator Tree:
                 TableScan
                   alias: src
@@ -2782,29 +2761,6 @@ STAGE PLANS:
                         value expressions: _col0 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 7 
-            Map Operator Tree:
-                TableScan
-                  alias: masking_test_subq_n3
-                  filterExpr: (UDFToDouble(key) is not null and key is not null) (type: boolean)
-                  Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: (UDFToDouble(key) is not null and key is not null) (type: boolean)
-                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                    Group By Operator
-                      keys: key (type: int)
-                      minReductionHashAggr: 0.5
-                      mode: hash
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 250 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: int)
-                        null sort order: z
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 250 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
@@ -2849,6 +2805,25 @@ STAGE PLANS:
                     Map-reduce partition columns: _col2 (type: double)
                     Statistics: Num rows: 325 Data size: 33475 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col0 (type: int), _col1 (type: string)
+        Reducer 5 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                keys: KEY._col0 (type: int)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 250 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
+                Select Operator
+                  expressions: _col0 (type: int), UDFToDouble(_col0) (type: double)
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 250 Data size: 3000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Reduce Output Operator
+                    key expressions: _col1 (type: double)
+                    null sort order: z
+                    sort order: +
+                    Map-reduce partition columns: _col1 (type: double)
+                    Statistics: Num rows: 250 Data size: 3000 Basic stats: COMPLETE Column stats: COMPLETE
+                    value expressions: _col0 (type: int)
         Reducer 6 
             Execution mode: llap
             Reduce Operator Tree:
@@ -2876,25 +2851,6 @@ STAGE PLANS:
                       sort order: ++
                       Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col1 (type: int)
                       Statistics: Num rows: 197 Data size: 17927 Basic stats: COMPLETE Column stats: COMPLETE
-        Reducer 8 
-            Execution mode: vectorized, llap
-            Reduce Operator Tree:
-              Group By Operator
-                keys: KEY._col0 (type: int)
-                mode: mergepartial
-                outputColumnNames: _col0
-                Statistics: Num rows: 250 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: _col0 (type: int), UDFToDouble(_col0) (type: double)
-                  outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 250 Data size: 3000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    key expressions: _col1 (type: double)
-                    null sort order: z
-                    sort order: +
-                    Map-reduce partition columns: _col1 (type: double)
-                    Statistics: Num rows: 250 Data size: 3000 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col0 (type: int)
 
   Stage: Stage-0
     Fetch Operator
@@ -7054,8 +7010,8 @@ STAGE PLANS:
 #### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-        Reducer 4 <- Map 3 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
-        Reducer 6 <- Map 5 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE)
+        Reducer 4 <- Map 5 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -7078,9 +7034,21 @@ STAGE PLANS:
                         Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col0 (type: int)
                         Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: string)
+                    Group By Operator
+                      keys: key (type: int)
+                      minReductionHashAggr: 0.5
+                      mode: hash
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 250 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 250 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 3 
+        Map 5 
             Map Operator Tree:
                 TableScan
                   alias: src
@@ -7102,29 +7070,6 @@ STAGE PLANS:
                         value expressions: _col0 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 5 
-            Map Operator Tree:
-                TableScan
-                  alias: masking_test_subq_n3
-                  filterExpr: ((key > 0) and UDFToDouble(key) is not null) (type: boolean)
-                  Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: ((key > 0) and UDFToDouble(key) is not null) (type: boolean)
-                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                    Group By Operator
-                      keys: key (type: int)
-                      minReductionHashAggr: 0.5
-                      mode: hash
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 250 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: int)
-                        null sort order: z
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 250 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
@@ -7143,6 +7088,25 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Reducer 3 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                keys: KEY._col0 (type: int)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 250 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
+                Select Operator
+                  expressions: _col0 (type: int), UDFToDouble(_col0) (type: double)
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 250 Data size: 3000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Reduce Output Operator
+                    key expressions: _col1 (type: double)
+                    null sort order: z
+                    sort order: +
+                    Map-reduce partition columns: _col1 (type: double)
+                    Statistics: Num rows: 250 Data size: 3000 Basic stats: COMPLETE Column stats: COMPLETE
+                    value expressions: _col0 (type: int)
         Reducer 4 
             Execution mode: llap
             Reduce Operator Tree:
@@ -7170,25 +7134,6 @@ STAGE PLANS:
                       sort order: ++
                       Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col1 (type: int)
                       Statistics: Num rows: 197 Data size: 17927 Basic stats: COMPLETE Column stats: COMPLETE
-        Reducer 6 
-            Execution mode: vectorized, llap
-            Reduce Operator Tree:
-              Group By Operator
-                keys: KEY._col0 (type: int)
-                mode: mergepartial
-                outputColumnNames: _col0
-                Statistics: Num rows: 250 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: _col0 (type: int), UDFToDouble(_col0) (type: double)
-                  outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 250 Data size: 3000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    key expressions: _col1 (type: double)
-                    null sort order: z
-                    sort order: +
-                    Map-reduce partition columns: _col1 (type: double)
-                    Statistics: Num rows: 250 Data size: 3000 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col0 (type: int)
 
   Stage: Stage-0
     Fetch Operator
@@ -7723,8 +7668,8 @@ STAGE PLANS:
 #### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-        Reducer 4 <- Map 3 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
-        Reducer 6 <- Map 5 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE)
+        Reducer 4 <- Map 5 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -7747,9 +7692,21 @@ STAGE PLANS:
                         Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col0 (type: int)
                         Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: string)
+                    Group By Operator
+                      keys: key (type: int)
+                      minReductionHashAggr: 0.5
+                      mode: hash
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 250 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 250 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 3 
+        Map 5 
             Map Operator Tree:
                 TableScan
                   alias: src
@@ -7771,29 +7728,6 @@ STAGE PLANS:
                         value expressions: _col0 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 5 
-            Map Operator Tree:
-                TableScan
-                  alias: masking_test_subq_n3
-                  filterExpr: ((key > 0) and UDFToDouble(key) is not null) (type: boolean)
-                  Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: ((key > 0) and UDFToDouble(key) is not null) (type: boolean)
-                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                    Group By Operator
-                      keys: key (type: int)
-                      minReductionHashAggr: 0.5
-                      mode: hash
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 250 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: int)
-                        null sort order: z
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 250 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
@@ -7812,6 +7746,25 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Reducer 3 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                keys: KEY._col0 (type: int)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 250 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
+                Select Operator
+                  expressions: _col0 (type: int), UDFToDouble(_col0) (type: double)
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 250 Data size: 3000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Reduce Output Operator
+                    key expressions: _col1 (type: double)
+                    null sort order: z
+                    sort order: +
+                    Map-reduce partition columns: _col1 (type: double)
+                    Statistics: Num rows: 250 Data size: 3000 Basic stats: COMPLETE Column stats: COMPLETE
+                    value expressions: _col0 (type: int)
         Reducer 4 
             Execution mode: llap
             Reduce Operator Tree:
@@ -7839,25 +7792,6 @@ STAGE PLANS:
                       sort order: ++
                       Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col1 (type: int)
                       Statistics: Num rows: 197 Data size: 17927 Basic stats: COMPLETE Column stats: COMPLETE
-        Reducer 6 
-            Execution mode: vectorized, llap
-            Reduce Operator Tree:
-              Group By Operator
-                keys: KEY._col0 (type: int)
-                mode: mergepartial
-                outputColumnNames: _col0
-                Statistics: Num rows: 250 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: _col0 (type: int), UDFToDouble(_col0) (type: double)
-                  outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 250 Data size: 3000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    key expressions: _col1 (type: double)
-                    null sort order: z
-                    sort order: +
-                    Map-reduce partition columns: _col1 (type: double)
-                    Statistics: Num rows: 250 Data size: 3000 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col0 (type: int)
 
   Stage: Stage-0
     Fetch Operator
diff --git a/ql/src/test/results/clientpositive/llap/masking_4.q.out b/ql/src/test/results/clientpositive/llap/masking_4.q.out
index 8e05859..ddd59dc 100644
--- a/ql/src/test/results/clientpositive/llap/masking_4.q.out
+++ b/ql/src/test/results/clientpositive/llap/masking_4.q.out
@@ -138,8 +138,8 @@ STAGE PLANS:
 #### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-        Reducer 4 <- Map 3 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
-        Reducer 6 <- Map 5 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE)
+        Reducer 4 <- Map 5 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -162,9 +162,21 @@ STAGE PLANS:
                         Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col0 (type: int)
                         Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: string)
+                    Group By Operator
+                      keys: key (type: int)
+                      minReductionHashAggr: 0.5
+                      mode: hash
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 250 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 250 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 3 
+        Map 5 
             Map Operator Tree:
                 TableScan
                   alias: src
@@ -186,29 +198,6 @@ STAGE PLANS:
                         value expressions: _col0 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 5 
-            Map Operator Tree:
-                TableScan
-                  alias: masking_test_subq_n2
-                  filterExpr: (UDFToDouble(key) is not null and key is not null) (type: boolean)
-                  Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: (UDFToDouble(key) is not null and key is not null) (type: boolean)
-                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                    Group By Operator
-                      keys: key (type: int)
-                      minReductionHashAggr: 0.5
-                      mode: hash
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 250 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: int)
-                        null sort order: z
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 250 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
@@ -227,6 +216,25 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Reducer 3 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                keys: KEY._col0 (type: int)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 250 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
+                Select Operator
+                  expressions: _col0 (type: int), UDFToDouble(_col0) (type: double)
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 250 Data size: 3000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Reduce Output Operator
+                    key expressions: _col1 (type: double)
+                    null sort order: z
+                    sort order: +
+                    Map-reduce partition columns: _col1 (type: double)
+                    Statistics: Num rows: 250 Data size: 3000 Basic stats: COMPLETE Column stats: COMPLETE
+                    value expressions: _col0 (type: int)
         Reducer 4 
             Execution mode: llap
             Reduce Operator Tree:
@@ -254,25 +262,6 @@ STAGE PLANS:
                       sort order: ++
                       Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col1 (type: int)
                       Statistics: Num rows: 197 Data size: 17927 Basic stats: COMPLETE Column stats: COMPLETE
-        Reducer 6 
-            Execution mode: vectorized, llap
-            Reduce Operator Tree:
-              Group By Operator
-                keys: KEY._col0 (type: int)
-                mode: mergepartial
-                outputColumnNames: _col0
-                Statistics: Num rows: 250 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: _col0 (type: int), UDFToDouble(_col0) (type: double)
-                  outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 250 Data size: 3000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    key expressions: _col1 (type: double)
-                    null sort order: z
-                    sort order: +
-                    Map-reduce partition columns: _col1 (type: double)
-                    Statistics: Num rows: 250 Data size: 3000 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col0 (type: int)
 
   Stage: Stage-0
     Fetch Operator
diff --git a/ql/src/test/results/clientpositive/llap/nonblock_op_deduplicate.q.out b/ql/src/test/results/clientpositive/llap/nonblock_op_deduplicate.q.out
index f97fb64..4c2614d 100644
--- a/ql/src/test/results/clientpositive/llap/nonblock_op_deduplicate.q.out
+++ b/ql/src/test/results/clientpositive/llap/nonblock_op_deduplicate.q.out
@@ -55,8 +55,8 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 4 (XPROD_EDGE)
-        Reducer 4 <- Map 3 (CUSTOM_SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 3 (XPROD_EDGE)
+        Reducer 3 <- Map 1 (CUSTOM_SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -73,13 +73,6 @@ STAGE PLANS:
                       sort order: 
                       Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
                       value expressions: _col0 (type: string), _col1 (type: string)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 3 
-            Map Operator Tree:
-                TableScan
-                  alias: src1
-                  Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
@@ -113,7 +106,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 4 
+        Reducer 3 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
diff --git a/ql/src/test/results/clientpositive/llap/ppd_join5.q.out b/ql/src/test/results/clientpositive/llap/ppd_join5.q.out
index f6346fd..91d8787 100644
--- a/ql/src/test/results/clientpositive/llap/ppd_join5.q.out
+++ b/ql/src/test/results/clientpositive/llap/ppd_join5.q.out
@@ -59,7 +59,7 @@ STAGE PLANS:
 #### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (XPROD_EDGE), Map 4 (XPROD_EDGE)
-        Reducer 3 <- Map 5 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+        Reducer 3 <- Map 4 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -86,7 +86,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: c
-                  filterExpr: (d <= 1) (type: boolean)
+                  filterExpr: ((d <= 1) or ((d <= 1) and id is not null)) (type: boolean)
                   Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (d <= 1) (type: boolean)
@@ -100,14 +100,6 @@ STAGE PLANS:
                         sort order: 
                         Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 5 
-            Map Operator Tree:
-                TableScan
-                  alias: b
-                  filterExpr: ((d <= 1) and id is not null) (type: boolean)
-                  Statistics: Num rows: 1 Data size: 89 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((d <= 1) and id is not null) (type: boolean)
                     Statistics: Num rows: 1 Data size: 89 Basic stats: COMPLETE Column stats: COMPLETE
@@ -202,7 +194,7 @@ STAGE PLANS:
 #### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
-        Reducer 3 <- Map 5 (XPROD_EDGE), Reducer 2 (XPROD_EDGE)
+        Reducer 3 <- Map 4 (XPROD_EDGE), Reducer 2 (XPROD_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -230,7 +222,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: b
-                  filterExpr: ((d <= 1) and id is not null) (type: boolean)
+                  filterExpr: (((d <= 1) and id is not null) or (d <= 1)) (type: boolean)
                   Statistics: Num rows: 1 Data size: 89 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((d <= 1) and id is not null) (type: boolean)
@@ -246,14 +238,6 @@ STAGE PLANS:
                         Map-reduce partition columns: _col0 (type: string), _col0 (type: string)
                         Statistics: Num rows: 1 Data size: 89 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: int)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 5 
-            Map Operator Tree:
-                TableScan
-                  alias: c
-                  filterExpr: (d <= 1) (type: boolean)
-                  Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (d <= 1) (type: boolean)
                     Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/ptf.q.out b/ql/src/test/results/clientpositive/llap/ptf.q.out
index a359017..37f838c 100644
--- a/ql/src/test/results/clientpositive/llap/ptf.q.out
+++ b/ql/src/test/results/clientpositive/llap/ptf.q.out
@@ -1111,29 +1111,14 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE)
-        Reducer 3 <- Map 4 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
-                  alias: part
-                  Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    key expressions: p_mfgr (type: string), p_name (type: string)
-                    null sort order: az
-                    sort order: ++
-                    Map-reduce partition columns: p_mfgr (type: string)
-                    Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 4 
-            Map Operator Tree:
-                TableScan
                   alias: p1
-                  filterExpr: p_partkey is not null (type: boolean)
                   Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: p_partkey is not null (type: boolean)
@@ -1144,11 +1129,36 @@ STAGE PLANS:
                       sort order: +
                       Map-reduce partition columns: p_partkey (type: int)
                       Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
+                  Reduce Output Operator
+                    key expressions: p_mfgr (type: string), p_name (type: string)
+                    null sort order: az
+                    sort order: ++
+                    Map-reduce partition columns: p_mfgr (type: string)
+                    Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
+                    value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: int)
+                  1 p_partkey (type: int)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
+                Statistics: Num rows: 27 Data size: 16713 Basic stats: COMPLETE Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 27 Data size: 16713 Basic stats: COMPLETE Column stats: COMPLETE
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Reducer 3 
+            Execution mode: llap
+            Reduce Operator Tree:
               Select Operator
                 expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -1177,24 +1187,6 @@ STAGE PLANS:
                       Map-reduce partition columns: _col0 (type: int)
                       Statistics: Num rows: 26 Data size: 23062 Basic stats: COMPLETE Column stats: COMPLETE
                       value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
-        Reducer 3 
-            Execution mode: llap
-            Reduce Operator Tree:
-              Merge Join Operator
-                condition map:
-                     Inner Join 0 to 1
-                keys:
-                  0 _col0 (type: int)
-                  1 p_partkey (type: int)
-                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
-                Statistics: Num rows: 27 Data size: 16713 Basic stats: COMPLETE Column stats: COMPLETE
-                File Output Operator
-                  compressed: false
-                  Statistics: Num rows: 27 Data size: 16713 Basic stats: COMPLETE Column stats: COMPLETE
-                  table:
-                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
 
   Stage: Stage-0
     Fetch Operator
@@ -1273,15 +1265,14 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-        Reducer 4 <- Map 3 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: p1
-                  filterExpr: p_partkey is not null (type: boolean)
                   Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: p_partkey is not null (type: boolean)
@@ -1292,13 +1283,6 @@ STAGE PLANS:
                       sort order: +
                       Map-reduce partition columns: p_partkey (type: int)
                       Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 3 
-            Map Operator Tree:
-                TableScan
-                  alias: part
-                  Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: p_mfgr (type: string), p_name (type: string)
                     null sort order: az
@@ -1330,7 +1314,7 @@ STAGE PLANS:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                         serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 4 
+        Reducer 3 
             Execution mode: llap
             Reduce Operator Tree:
               Select Operator
@@ -2442,30 +2426,15 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE)
-        Reducer 3 <- Map 5 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
-        Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+        Reducer 4 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
-                  alias: part
-                  Statistics: Num rows: 26 Data size: 6110 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    key expressions: p_mfgr (type: string), p_name (type: string)
-                    null sort order: az
-                    sort order: ++
-                    Map-reduce partition columns: p_mfgr (type: string)
-                    Statistics: Num rows: 26 Data size: 6110 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: p_partkey (type: int), p_size (type: int), p_retailprice (type: double)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 5 
-            Map Operator Tree:
-                TableScan
                   alias: p1
-                  filterExpr: p_partkey is not null (type: boolean)
                   Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: p_partkey is not null (type: boolean)
@@ -2476,42 +2445,18 @@ STAGE PLANS:
                       sort order: +
                       Map-reduce partition columns: p_partkey (type: int)
                       Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
+                  Reduce Output Operator
+                    key expressions: p_mfgr (type: string), p_name (type: string)
+                    null sort order: az
+                    sort order: ++
+                    Map-reduce partition columns: p_mfgr (type: string)
+                    Statistics: Num rows: 26 Data size: 6110 Basic stats: COMPLETE Column stats: COMPLETE
+                    value expressions: p_partkey (type: int), p_size (type: int), p_retailprice (type: double)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
-              Select Operator
-                expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col3 (type: int), VALUE._col5 (type: double)
-                outputColumnNames: _col0, _col1, _col2, _col5, _col7
-                Statistics: Num rows: 26 Data size: 13078 Basic stats: COMPLETE Column stats: COMPLETE
-                PTF Operator
-                  Function definitions:
-                      Input definition
-                        input alias: part
-                        output shape: _col0: int, _col1: string, _col2: string, _col5: int, _col7: double
-                        type: TABLE
-                      Partition table definition
-                        input alias: abc
-                        name: noop
-                        order by: _col1 ASC NULLS LAST
-                        output shape: _col0: int, _col1: string, _col2: string, _col5: int, _col7: double
-                        partition by: _col2
-                        raw input shape:
-                  Statistics: Num rows: 26 Data size: 13078 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: _col0 is not null (type: boolean)
-                    Statistics: Num rows: 26 Data size: 13078 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      null sort order: z
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
-                      Statistics: Num rows: 26 Data size: 13078 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: _col1 (type: string), _col2 (type: string), _col5 (type: int), _col7 (type: double)
-        Reducer 3 
-            Execution mode: llap
-            Reduce Operator Tree:
               Merge Join Operator
                 condition map:
                      Inner Join 0 to 1
@@ -2527,7 +2472,7 @@ STAGE PLANS:
                   Map-reduce partition columns: _col2 (type: string)
                   Statistics: Num rows: 27 Data size: 6237 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col5 (type: int), _col7 (type: double)
-        Reducer 4 
+        Reducer 3 
             Execution mode: llap
             Reduce Operator Tree:
               Select Operator
@@ -2592,6 +2537,37 @@ STAGE PLANS:
                           input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                           output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                           serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Reducer 4 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Select Operator
+                expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col3 (type: int), VALUE._col5 (type: double)
+                outputColumnNames: _col0, _col1, _col2, _col5, _col7
+                Statistics: Num rows: 26 Data size: 13078 Basic stats: COMPLETE Column stats: COMPLETE
+                PTF Operator
+                  Function definitions:
+                      Input definition
+                        input alias: part
+                        output shape: _col0: int, _col1: string, _col2: string, _col5: int, _col7: double
+                        type: TABLE
+                      Partition table definition
+                        input alias: abc
+                        name: noop
+                        order by: _col1 ASC NULLS LAST
+                        output shape: _col0: int, _col1: string, _col2: string, _col5: int, _col7: double
+                        partition by: _col2
+                        raw input shape:
+                  Statistics: Num rows: 26 Data size: 13078 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: _col0 is not null (type: boolean)
+                    Statistics: Num rows: 26 Data size: 13078 Basic stats: COMPLETE Column stats: COMPLETE
+                    Reduce Output Operator
+                      key expressions: _col0 (type: int)
+                      null sort order: z
+                      sort order: +
+                      Map-reduce partition columns: _col0 (type: int)
+                      Statistics: Num rows: 26 Data size: 13078 Basic stats: COMPLETE Column stats: COMPLETE
+                      value expressions: _col1 (type: string), _col2 (type: string), _col5 (type: int), _col7 (type: double)
 
   Stage: Stage-0
     Fetch Operator
diff --git a/ql/src/test/results/clientpositive/llap/ptf_streaming.q.out b/ql/src/test/results/clientpositive/llap/ptf_streaming.q.out
index 06ac598..6580261 100644
--- a/ql/src/test/results/clientpositive/llap/ptf_streaming.q.out
+++ b/ql/src/test/results/clientpositive/llap/ptf_streaming.q.out
@@ -414,29 +414,14 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE)
-        Reducer 3 <- Map 4 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
-                  alias: part
-                  Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    key expressions: p_mfgr (type: string), p_name (type: string)
-                    null sort order: az
-                    sort order: ++
-                    Map-reduce partition columns: p_mfgr (type: string)
-                    Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string)
-            Execution mode: llap
-            LLAP IO: no inputs
-        Map 4 
-            Map Operator Tree:
-                TableScan
                   alias: p1
-                  filterExpr: p_partkey is not null (type: boolean)
                   Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: p_partkey is not null (type: boolean)
@@ -447,11 +432,36 @@ STAGE PLANS:
                       sort order: +
                       Map-reduce partition columns: p_partkey (type: int)
                       Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
+                  Reduce Output Operator
+                    key expressions: p_mfgr (type: string), p_name (type: string)
+                    null sort order: az
+                    sort order: ++
+                    Map-reduce partition columns: p_mfgr (type: string)
+                    Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
+                    value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string)
             Execution mode: llap
             LLAP IO: no inputs
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: int)
+                  1 p_partkey (type: int)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
+                Statistics: Num rows: 27 Data size: 16713 Basic stats: COMPLETE Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 27 Data size: 16713 Basic stats: COMPLETE Column stats: COMPLETE
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Reducer 3 
+            Execution mode: llap
+            Reduce Operator Tree:
               Select Operator
                 expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -480,24 +490,6 @@ STAGE PLANS:
                       Map-reduce partition columns: _col0 (type: int)
                       Statistics: Num rows: 26 Data size: 23062 Basic stats: COMPLETE Column stats: COMPLETE
                       value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
-        Reducer 3 
-            Execution mode: llap
-            Reduce Operator Tree:
-              Merge Join Operator
-                condition map:
-                     Inner Join 0 to 1
-                keys:
-                  0 _col0 (type: int)
-                  1 p_partkey (type: int)
-                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
-                Statistics: Num rows: 27 Data size: 16713 Basic stats: COMPLETE Column stats: COMPLETE
-                File Output Operator
-                  compressed: false
-                  Statistics: Num rows: 27 Data size: 16713 Basic stats: COMPLETE Column stats: COMPLETE
-                  table:
-                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
 
   Stage: Stage-0
     Fetch Operator
@@ -1699,30 +1691,15 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE)
-        Reducer 3 <- Map 5 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
-        Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+        Reducer 4 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
-                  alias: part
-                  Statistics: Num rows: 26 Data size: 6110 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    key expressions: p_mfgr (type: string), p_name (type: string)
-                    null sort order: az
-                    sort order: ++
-                    Map-reduce partition columns: p_mfgr (type: string)
-                    Statistics: Num rows: 26 Data size: 6110 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: p_partkey (type: int), p_size (type: int), p_retailprice (type: double)
-            Execution mode: llap
-            LLAP IO: no inputs
-        Map 5 
-            Map Operator Tree:
-                TableScan
                   alias: p1
-                  filterExpr: p_partkey is not null (type: boolean)
                   Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: p_partkey is not null (type: boolean)
@@ -1733,42 +1710,18 @@ STAGE PLANS:
                       sort order: +
                       Map-reduce partition columns: p_partkey (type: int)
                       Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
+                  Reduce Output Operator
+                    key expressions: p_mfgr (type: string), p_name (type: string)
+                    null sort order: az
+                    sort order: ++
+                    Map-reduce partition columns: p_mfgr (type: string)
+                    Statistics: Num rows: 26 Data size: 6110 Basic stats: COMPLETE Column stats: COMPLETE
+                    value expressions: p_partkey (type: int), p_size (type: int), p_retailprice (type: double)
             Execution mode: llap
             LLAP IO: no inputs
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
-              Select Operator
-                expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col3 (type: int), VALUE._col5 (type: double)
-                outputColumnNames: _col0, _col1, _col2, _col5, _col7
-                Statistics: Num rows: 26 Data size: 13078 Basic stats: COMPLETE Column stats: COMPLETE
-                PTF Operator
-                  Function definitions:
-                      Input definition
-                        input alias: part
-                        output shape: _col0: int, _col1: string, _col2: string, _col5: int, _col7: double
-                        type: TABLE
-                      Partition table definition
-                        input alias: abc
-                        name: noopstreaming
-                        order by: _col1 ASC NULLS LAST
-                        output shape: _col0: int, _col1: string, _col2: string, _col5: int, _col7: double
-                        partition by: _col2
-                        raw input shape:
-                  Statistics: Num rows: 26 Data size: 13078 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: _col0 is not null (type: boolean)
-                    Statistics: Num rows: 26 Data size: 13078 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      null sort order: z
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
-                      Statistics: Num rows: 26 Data size: 13078 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: _col1 (type: string), _col2 (type: string), _col5 (type: int), _col7 (type: double)
-        Reducer 3 
-            Execution mode: llap
-            Reduce Operator Tree:
               Merge Join Operator
                 condition map:
                      Inner Join 0 to 1
@@ -1784,7 +1737,7 @@ STAGE PLANS:
                   Map-reduce partition columns: _col2 (type: string)
                   Statistics: Num rows: 27 Data size: 6237 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col5 (type: int), _col7 (type: double)
-        Reducer 4 
+        Reducer 3 
             Execution mode: llap
             Reduce Operator Tree:
               Select Operator
@@ -1849,6 +1802,37 @@ STAGE PLANS:
                           input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                           output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                           serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Reducer 4 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Select Operator
+                expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col3 (type: int), VALUE._col5 (type: double)
+                outputColumnNames: _col0, _col1, _col2, _col5, _col7
+                Statistics: Num rows: 26 Data size: 13078 Basic stats: COMPLETE Column stats: COMPLETE
+                PTF Operator
+                  Function definitions:
+                      Input definition
+                        input alias: part
+                        output shape: _col0: int, _col1: string, _col2: string, _col5: int, _col7: double
+                        type: TABLE
+                      Partition table definition
+                        input alias: abc
+                        name: noopstreaming
+                        order by: _col1 ASC NULLS LAST
+                        output shape: _col0: int, _col1: string, _col2: string, _col5: int, _col7: double
+                        partition by: _col2
+                        raw input shape:
+                  Statistics: Num rows: 26 Data size: 13078 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: _col0 is not null (type: boolean)
+                    Statistics: Num rows: 26 Data size: 13078 Basic stats: COMPLETE Column stats: COMPLETE
+                    Reduce Output Operator
+                      key expressions: _col0 (type: int)
+                      null sort order: z
+                      sort order: +
+                      Map-reduce partition columns: _col0 (type: int)
+                      Statistics: Num rows: 26 Data size: 13078 Basic stats: COMPLETE Column stats: COMPLETE
+                      value expressions: _col1 (type: string), _col2 (type: string), _col5 (type: int), _col7 (type: double)
 
   Stage: Stage-0
     Fetch Operator
diff --git a/ql/src/test/results/clientpositive/llap/reduce_deduplicate_extended2.q.out b/ql/src/test/results/clientpositive/llap/reduce_deduplicate_extended2.q.out
index b8d49c0..c985276 100644
--- a/ql/src/test/results/clientpositive/llap/reduce_deduplicate_extended2.q.out
+++ b/ql/src/test/results/clientpositive/llap/reduce_deduplicate_extended2.q.out
@@ -553,9 +553,9 @@ STAGE PLANS:
 #### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (XPROD_EDGE), Reducer 6 (XPROD_EDGE)
+        Reducer 3 <- Reducer 2 (XPROD_EDGE), Reducer 5 (XPROD_EDGE)
         Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
-        Reducer 6 <- Map 5 (SIMPLE_EDGE)
+        Reducer 5 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -579,13 +579,6 @@ STAGE PLANS:
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 250 Data size: 21750 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 5 
-            Map Operator Tree:
-                TableScan
-                  alias: src
-                  Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: value (type: string)
                     outputColumnNames: value
@@ -647,7 +640,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 6 
+        Reducer 5 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
diff --git a/ql/src/test/results/clientpositive/llap/semijoin.q.out b/ql/src/test/results/clientpositive/llap/semijoin.q.out
index cc58789..24f6f52 100644
--- a/ql/src/test/results/clientpositive/llap/semijoin.q.out
+++ b/ql/src/test/results/clientpositive/llap/semijoin.q.out
@@ -3346,15 +3346,15 @@ STAGE PLANS:
 #### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
-        Reducer 3 <- Map 1 (XPROD_EDGE), Reducer 5 (XPROD_EDGE)
-        Reducer 5 <- Map 4 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (XPROD_EDGE), Reducer 4 (XPROD_EDGE)
+        Reducer 4 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: outr
-                  filterExpr: (key is not null and value is not null) (type: boolean)
+                  filterExpr: ((key is not null and value is not null) or value is not null) (type: boolean)
                   Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key is not null and value is not null) (type: boolean)
@@ -3374,14 +3374,6 @@ STAGE PLANS:
                         sort order: 
                         Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: string), _col1 (type: string)
-            Execution mode: llap
-            LLAP IO: no inputs
-        Map 4 
-            Map Operator Tree:
-                TableScan
-                  alias: src
-                  filterExpr: value is not null (type: boolean)
-                  Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: value is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
@@ -3445,7 +3437,7 @@ STAGE PLANS:
                       sort order: ++
                       Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
                       Statistics: Num rows: 20833 Data size: 3708274 Basic stats: COMPLETE Column stats: COMPLETE
-        Reducer 5 
+        Reducer 4 
             Execution mode: llap
             Reduce Operator Tree:
               Group By Operator
diff --git a/ql/src/test/results/clientpositive/llap/semijoin2.q.out b/ql/src/test/results/clientpositive/llap/semijoin2.q.out
index 94020e0..a3a04a7 100644
--- a/ql/src/test/results/clientpositive/llap/semijoin2.q.out
+++ b/ql/src/test/results/clientpositive/llap/semijoin2.q.out
@@ -61,54 +61,18 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 6 (SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
         Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
         Reducer 5 <- Reducer 4 (SIMPLE_EDGE)
-        Reducer 8 <- Map 7 (SIMPLE_EDGE), Map 9 (SIMPLE_EDGE)
+        Reducer 6 <- Map 1 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
-                  alias: t1
-                  filterExpr: (bigint_col_22 is not null and decimal1709_col_26 is not null and tinyint_col_8 is not null and timestamp_col_10 is not null) (type: boolean)
-                  Statistics: Num rows: 1 Data size: 176 Basic stats: COMPLETE Column stats: NONE
-                  Filter Operator
-                    predicate: (bigint_col_22 is not null and decimal1709_col_26 is not null and tinyint_col_8 is not null and timestamp_col_10 is not null) (type: boolean)
-                    Statistics: Num rows: 1 Data size: 176 Basic stats: COMPLETE Column stats: NONE
-                    Reduce Output Operator
-                      key expressions: bigint_col_22 (type: bigint), decimal1709_col_26 (type: decimal(38,23)), tinyint_col_8 (type: tinyint)
-                      null sort order: zzz
-                      sort order: +++
-                      Map-reduce partition columns: bigint_col_22 (type: bigint), decimal1709_col_26 (type: decimal(38,23)), tinyint_col_8 (type: tinyint)
-                      Statistics: Num rows: 1 Data size: 176 Basic stats: COMPLETE Column stats: NONE
-                      value expressions: smallint_col_25 (type: smallint), double_col_61 (type: double), timestamp_col_10 (type: timestamp)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 6 
-            Map Operator Tree:
-                TableScan
-                  alias: t2
-                  filterExpr: (UDFToLong(tinyint_col_6) is not null and decimal0504_col_37 is not null and tinyint_col_33 is not null and UDFToInteger(smallint_col_38) is not null) (type: boolean)
-                  Statistics: Num rows: 1 Data size: 128 Basic stats: COMPLETE Column stats: NONE
-                  Filter Operator
-                    predicate: (UDFToLong(tinyint_col_6) is not null and decimal0504_col_37 is not null and tinyint_col_33 is not null and UDFToInteger(smallint_col_38) is not null) (type: boolean)
-                    Statistics: Num rows: 1 Data size: 128 Basic stats: COMPLETE Column stats: NONE
-                    Reduce Output Operator
-                      key expressions: UDFToLong(tinyint_col_6) (type: bigint), decimal0504_col_37 (type: decimal(38,23)), tinyint_col_33 (type: tinyint)
-                      null sort order: zzz
-                      sort order: +++
-                      Map-reduce partition columns: UDFToLong(tinyint_col_6) (type: bigint), decimal0504_col_37 (type: decimal(38,23)), tinyint_col_33 (type: tinyint)
-                      Statistics: Num rows: 1 Data size: 128 Basic stats: COMPLETE Column stats: NONE
-                      value expressions: int_col_2 (type: int), smallint_col_38 (type: smallint)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 7 
-            Map Operator Tree:
-                TableScan
                   alias: tt1
-                  filterExpr: decimal1208_col_20 is not null (type: boolean)
+                  filterExpr: (decimal1208_col_20 is not null or (bigint_col_22 is not null and decimal1709_col_26 is not null and tinyint_col_8 is not null and timestamp_col_10 is not null)) (type: boolean)
                   Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
                     predicate: decimal1208_col_20 is not null (type: boolean)
@@ -119,13 +83,23 @@ STAGE PLANS:
                       sort order: +
                       Map-reduce partition columns: decimal1208_col_20 (type: decimal(38,6))
                       Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: (bigint_col_22 is not null and decimal1709_col_26 is not null and tinyint_col_8 is not null and timestamp_col_10 is not null) (type: boolean)
+                    Statistics: Num rows: 1 Data size: 176 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: bigint_col_22 (type: bigint), decimal1709_col_26 (type: decimal(38,23)), tinyint_col_8 (type: tinyint)
+                      null sort order: zzz
+                      sort order: +++
+                      Map-reduce partition columns: bigint_col_22 (type: bigint), decimal1709_col_26 (type: decimal(38,23)), tinyint_col_8 (type: tinyint)
+                      Statistics: Num rows: 1 Data size: 176 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: smallint_col_25 (type: smallint), double_col_61 (type: double), timestamp_col_10 (type: timestamp)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 9 
+        Map 7 
             Map Operator Tree:
                 TableScan
                   alias: tt2
-                  filterExpr: (decimal1611_col_22 is not null and timestamp_col_18 is not null) (type: boolean)
+                  filterExpr: ((decimal1611_col_22 is not null and timestamp_col_18 is not null) or (UDFToLong(tinyint_col_6) is not null and decimal0504_col_37 is not null and tinyint_col_33 is not null and UDFToInteger(smallint_col_38) is not null)) (type: boolean)
                   Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
                     predicate: (decimal1611_col_22 is not null and timestamp_col_18 is not null) (type: boolean)
@@ -137,6 +111,16 @@ STAGE PLANS:
                       Map-reduce partition columns: decimal1611_col_22 (type: decimal(38,6))
                       Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: NONE
                       value expressions: timestamp_col_18 (type: timestamp)
+                  Filter Operator
+                    predicate: (UDFToLong(tinyint_col_6) is not null and decimal0504_col_37 is not null and tinyint_col_33 is not null and UDFToInteger(smallint_col_38) is not null) (type: boolean)
+                    Statistics: Num rows: 1 Data size: 128 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: UDFToLong(tinyint_col_6) (type: bigint), decimal0504_col_37 (type: decimal(38,23)), tinyint_col_33 (type: tinyint)
+                      null sort order: zzz
+                      sort order: +++
+                      Map-reduce partition columns: UDFToLong(tinyint_col_6) (type: bigint), decimal0504_col_37 (type: decimal(38,23)), tinyint_col_33 (type: tinyint)
+                      Statistics: Num rows: 1 Data size: 128 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: int_col_2 (type: int), smallint_col_38 (type: smallint)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
@@ -146,17 +130,26 @@ STAGE PLANS:
                 condition map:
                      Inner Join 0 to 1
                 keys:
-                  0 bigint_col_22 (type: bigint), decimal1709_col_26 (type: decimal(38,23)), tinyint_col_8 (type: tinyint)
-                  1 UDFToLong(tinyint_col_6) (type: bigint), decimal0504_col_37 (type: decimal(38,23)), tinyint_col_33 (type: tinyint)
-                outputColumnNames: _col16, _col21, _col72, _col98, _col105
-                Statistics: Num rows: 1 Data size: 193 Basic stats: COMPLETE Column stats: NONE
-                Reduce Output Operator
-                  key expressions: UDFToInteger(_col105) (type: int), _col72 (type: timestamp)
-                  null sort order: zz
-                  sort order: ++
-                  Map-reduce partition columns: UDFToInteger(_col105) (type: int), _col72 (type: timestamp)
-                  Statistics: Num rows: 1 Data size: 193 Basic stats: COMPLETE Column stats: NONE
-                  value expressions: _col16 (type: smallint), _col21 (type: double), _col98 (type: int)
+                  0 decimal1208_col_20 (type: decimal(38,6))
+                  1 decimal1611_col_22 (type: decimal(38,6))
+                outputColumnNames: _col115
+                Statistics: Num rows: 1 Data size: 123 Basic stats: COMPLETE Column stats: NONE
+                Select Operator
+                  expressions: _col115 (type: timestamp)
+                  outputColumnNames: _col1
+                  Statistics: Num rows: 1 Data size: 123 Basic stats: COMPLETE Column stats: NONE
+                  Group By Operator
+                    keys: -92 (type: int), _col1 (type: timestamp)
+                    minReductionHashAggr: 0.99
+                    mode: hash
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 1 Data size: 123 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: -92 (type: int), _col1 (type: timestamp)
+                      null sort order: zz
+                      sort order: ++
+                      Map-reduce partition columns: -92 (type: int), _col1 (type: timestamp)
+                      Statistics: Num rows: 1 Data size: 123 Basic stats: COMPLETE Column stats: NONE
         Reducer 3 
             Execution mode: llap
             Reduce Operator Tree:
@@ -252,33 +245,24 @@ STAGE PLANS:
                           input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                           output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                           serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 8 
+        Reducer 6 
             Execution mode: llap
             Reduce Operator Tree:
               Merge Join Operator
                 condition map:
                      Inner Join 0 to 1
                 keys:
-                  0 decimal1208_col_20 (type: decimal(38,6))
-                  1 decimal1611_col_22 (type: decimal(38,6))
-                outputColumnNames: _col115
-                Statistics: Num rows: 1 Data size: 123 Basic stats: COMPLETE Column stats: NONE
-                Select Operator
-                  expressions: _col115 (type: timestamp)
-                  outputColumnNames: _col1
-                  Statistics: Num rows: 1 Data size: 123 Basic stats: COMPLETE Column stats: NONE
-                  Group By Operator
-                    keys: -92 (type: int), _col1 (type: timestamp)
-                    minReductionHashAggr: 0.99
-                    mode: hash
-                    outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 1 Data size: 123 Basic stats: COMPLETE Column stats: NONE
-                    Reduce Output Operator
-                      key expressions: -92 (type: int), _col1 (type: timestamp)
-                      null sort order: zz
-                      sort order: ++
-                      Map-reduce partition columns: -92 (type: int), _col1 (type: timestamp)
-                      Statistics: Num rows: 1 Data size: 123 Basic stats: COMPLETE Column stats: NONE
+                  0 bigint_col_22 (type: bigint), decimal1709_col_26 (type: decimal(38,23)), tinyint_col_8 (type: tinyint)
+                  1 UDFToLong(tinyint_col_6) (type: bigint), decimal0504_col_37 (type: decimal(38,23)), tinyint_col_33 (type: tinyint)
+                outputColumnNames: _col16, _col21, _col72, _col98, _col105
+                Statistics: Num rows: 1 Data size: 193 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: UDFToInteger(_col105) (type: int), _col72 (type: timestamp)
+                  null sort order: zz
+                  sort order: ++
+                  Map-reduce partition columns: UDFToInteger(_col105) (type: int), _col72 (type: timestamp)
+                  Statistics: Num rows: 1 Data size: 193 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col16 (type: smallint), _col21 (type: double), _col98 (type: int)
 
   Stage: Stage-0
     Fetch Operator
diff --git a/ql/src/test/results/clientpositive/llap/semijoin4.q.out b/ql/src/test/results/clientpositive/llap/semijoin4.q.out
index 7b51a63..e13ade3 100644
--- a/ql/src/test/results/clientpositive/llap/semijoin4.q.out
+++ b/ql/src/test/results/clientpositive/llap/semijoin4.q.out
@@ -70,17 +70,17 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (XPROD_EDGE), Reducer 7 (XPROD_EDGE)
-        Reducer 6 <- Map 5 (SIMPLE_EDGE), Map 8 (SIMPLE_EDGE)
-        Reducer 7 <- Reducer 6 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 6 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (XPROD_EDGE), Reducer 5 (XPROD_EDGE)
+        Reducer 4 <- Map 1 (SIMPLE_EDGE), Map 6 (SIMPLE_EDGE)
+        Reducer 5 <- Reducer 4 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: t1
-                  filterExpr: ((tinyint_col_46 = -92Y) and bigint_col_13 is not null and decimal1309_col_65 is not null) (type: boolean)
+                  filterExpr: (((tinyint_col_46 = -92Y) and bigint_col_13 is not null and decimal1309_col_65 is not null) or (decimal1309_col_65 is not null and timestamp_col_66 is not null)) (type: boolean)
                   Statistics: Num rows: 1 Data size: 124 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
                     predicate: ((tinyint_col_46 = -92Y) and bigint_col_13 is not null and decimal1309_col_65 is not null) (type: boolean)
@@ -95,13 +95,26 @@ STAGE PLANS:
                         sort order: ++
                         Map-reduce partition columns: _col0 (type: bigint), _col1 (type: decimal(27,9))
                         Statistics: Num rows: 1 Data size: 124 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: (decimal1309_col_65 is not null and timestamp_col_66 is not null) (type: boolean)
+                    Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: decimal1309_col_65 (type: decimal(13,9)), timestamp_col_66 (type: timestamp)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: decimal(19,11)), _col1 (type: timestamp)
+                        null sort order: zz
+                        sort order: ++
+                        Map-reduce partition columns: _col0 (type: decimal(19,11)), _col1 (type: timestamp)
+                        Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: NONE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
-        Map 4 
+        Map 6 
             Map Operator Tree:
                 TableScan
                   alias: t2
-                  filterExpr: ((tinyint_col_21 = -92Y) and UDFToLong(tinyint_col_18) is not null and decimal2709_col_9 is not null) (type: boolean)
+                  filterExpr: (((tinyint_col_21 = -92Y) and UDFToLong(tinyint_col_18) is not null and decimal2709_col_9 is not null) or (decimal1911_col_16 is not null and timestamp_col_19 is not null)) (type: boolean)
                   Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
                     predicate: ((tinyint_col_21 = -92Y) and UDFToLong(tinyint_col_18) is not null and decimal2709_col_9 is not null) (type: boolean)
@@ -116,35 +129,6 @@ STAGE PLANS:
                         sort order: ++
                         Map-reduce partition columns: _col1 (type: bigint), _col0 (type: decimal(27,9))
                         Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: NONE
-            Execution mode: vectorized, llap
-            LLAP IO: all inputs
-        Map 5 
-            Map Operator Tree:
-                TableScan
-                  alias: tt1
-                  filterExpr: (decimal1309_col_65 is not null and timestamp_col_66 is not null) (type: boolean)
-                  Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: NONE
-                  Filter Operator
-                    predicate: (decimal1309_col_65 is not null and timestamp_col_66 is not null) (type: boolean)
-                    Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: decimal1309_col_65 (type: decimal(13,9)), timestamp_col_66 (type: timestamp)
-                      outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: NONE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: decimal(19,11)), _col1 (type: timestamp)
-                        null sort order: zz
-                        sort order: ++
-                        Map-reduce partition columns: _col0 (type: decimal(19,11)), _col1 (type: timestamp)
-                        Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: NONE
-            Execution mode: vectorized, llap
-            LLAP IO: all inputs
-        Map 8 
-            Map Operator Tree:
-                TableScan
-                  alias: tt2
-                  filterExpr: (decimal1911_col_16 is not null and timestamp_col_19 is not null) (type: boolean)
-                  Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
                     predicate: (decimal1911_col_16 is not null and timestamp_col_19 is not null) (type: boolean)
                     Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: NONE
@@ -195,7 +179,7 @@ STAGE PLANS:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                         serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 6 
+        Reducer 4 
             Execution mode: llap
             Reduce Operator Tree:
               Merge Join Operator
@@ -217,7 +201,7 @@ STAGE PLANS:
                     sort order: +
                     Map-reduce partition columns: _col0 (type: boolean)
                     Statistics: Num rows: 1 Data size: 167 Basic stats: COMPLETE Column stats: NONE
-        Reducer 7 
+        Reducer 5 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
diff --git a/ql/src/test/results/clientpositive/llap/semijoin5.q.out b/ql/src/test/results/clientpositive/llap/semijoin5.q.out
index 72184e3..a22e00a 100644
--- a/ql/src/test/results/clientpositive/llap/semijoin5.q.out
+++ b/ql/src/test/results/clientpositive/llap/semijoin5.q.out
@@ -61,17 +61,17 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 6 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
         Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
-        Reducer 7 <- Map 6 (SIMPLE_EDGE), Map 8 (SIMPLE_EDGE)
+        Reducer 5 <- Map 1 (SIMPLE_EDGE), Map 6 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: t1
-                  filterExpr: (bigint_col_7 is not null and decimal2016_col_26 is not null and tinyint_col_3 is not null and timestamp_col_9 is not null) (type: boolean)
+                  filterExpr: ((bigint_col_7 is not null and decimal2016_col_26 is not null and tinyint_col_3 is not null and timestamp_col_9 is not null) or decimal2612_col_77 is not null) (type: boolean)
                   Statistics: Num rows: 1 Data size: 176 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
                     predicate: (bigint_col_7 is not null and decimal2016_col_26 is not null and tinyint_col_3 is not null and timestamp_col_9 is not null) (type: boolean)
@@ -87,13 +87,26 @@ STAGE PLANS:
                         Map-reduce partition columns: _col0 (type: tinyint), _col4 (type: decimal(34,16)), _col1 (type: bigint)
                         Statistics: Num rows: 1 Data size: 176 Basic stats: COMPLETE Column stats: NONE
                         value expressions: _col2 (type: timestamp), _col3 (type: double), _col5 (type: smallint)
+                  Filter Operator
+                    predicate: decimal2612_col_77 is not null (type: boolean)
+                    Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: decimal2612_col_77 (type: decimal(26,12))
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: decimal(26,12))
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: decimal(26,12))
+                        Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
-        Map 5 
+        Map 6 
             Map Operator Tree:
                 TableScan
                   alias: t2
-                  filterExpr: (UDFToLong(tinyint_col_15) is not null and decimal2709_col_9 is not null and tinyint_col_20 is not null and UDFToInteger(smallint_col_19) is not null) (type: boolean)
+                  filterExpr: ((UDFToLong(tinyint_col_15) is not null and decimal2709_col_9 is not null and tinyint_col_20 is not null and UDFToInteger(smallint_col_19) is not null) or (timestamp_col_18 is not null and decimal1911_col_16 is not null)) (type: boolean)
                   Statistics: Num rows: 1 Data size: 128 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
                     predicate: (UDFToLong(tinyint_col_15) is not null and decimal2709_col_9 is not null and tinyint_col_20 is not null and UDFToInteger(smallint_col_19) is not null) (type: boolean)
@@ -109,14 +122,6 @@ STAGE PLANS:
                         Map-reduce partition columns: _col4 (type: tinyint), _col0 (type: decimal(34,16)), UDFToLong(_col2) (type: bigint)
                         Statistics: Num rows: 1 Data size: 128 Basic stats: COMPLETE Column stats: NONE
                         value expressions: _col1 (type: int), _col3 (type: smallint)
-            Execution mode: vectorized, llap
-            LLAP IO: all inputs
-        Map 6 
-            Map Operator Tree:
-                TableScan
-                  alias: tt2
-                  filterExpr: (timestamp_col_18 is not null and decimal1911_col_16 is not null) (type: boolean)
-                  Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
                     predicate: (timestamp_col_18 is not null and decimal1911_col_16 is not null) (type: boolean)
                     Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: NONE
@@ -133,27 +138,6 @@ STAGE PLANS:
                         value expressions: _col1 (type: timestamp)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
-        Map 8 
-            Map Operator Tree:
-                TableScan
-                  alias: tt1
-                  filterExpr: decimal2612_col_77 is not null (type: boolean)
-                  Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE
-                  Filter Operator
-                    predicate: decimal2612_col_77 is not null (type: boolean)
-                    Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: decimal2612_col_77 (type: decimal(26,12))
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: decimal(26,12))
-                        null sort order: z
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: decimal(26,12))
-                        Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE
-            Execution mode: vectorized, llap
-            LLAP IO: all inputs
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
@@ -228,7 +212,7 @@ STAGE PLANS:
                           input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                           output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                           serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 7 
+        Reducer 5 
             Execution mode: llap
             Reduce Operator Tree:
               Merge Join Operator
diff --git a/ql/src/test/results/clientpositive/llap/sharedwork.q.out b/ql/src/test/results/clientpositive/llap/sharedwork.q.out
index 22b1cee..f3630ba 100644
--- a/ql/src/test/results/clientpositive/llap/sharedwork.q.out
+++ b/ql/src/test/results/clientpositive/llap/sharedwork.q.out
@@ -618,12 +618,12 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 10 <- Map 8 (SIMPLE_EDGE)
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE)
-        Reducer 5 <- Map 4 (SIMPLE_EDGE), Map 8 (SIMPLE_EDGE)
-        Reducer 6 <- Map 4 (SIMPLE_EDGE)
-        Reducer 7 <- Reducer 10 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
+        Reducer 4 <- Map 1 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE)
+        Reducer 5 <- Map 1 (SIMPLE_EDGE)
+        Reducer 6 <- Reducer 5 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE)
+        Reducer 9 <- Map 7 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -647,52 +647,6 @@ STAGE PLANS:
                       tag: 0
                       value expressions: _col0 (type: string), _col1 (type: string)
                       auto parallelism: true
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-            Path -> Alias:
-#### A masked pattern was here ####
-            Path -> Partition:
-#### A masked pattern was here ####
-                Partition
-                  base file name: part
-                  input format: org.apache.hadoop.mapred.TextInputFormat
-                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                  properties:
-                    bucket_count -1
-                    bucketing_version 2
-                    column.name.delimiter ,
-                    columns p_partkey,p_name,p_mfgr,p_brand,p_type,p_size,p_container,p_retailprice,p_comment
-                    columns.types int:string:string:string:string:int:string:double:string
-#### A masked pattern was here ####
-                    name default.part
-                    serialization.format 1
-                    serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                
-                    input format: org.apache.hadoop.mapred.TextInputFormat
-                    output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                    properties:
-                      bucketing_version 2
-                      column.name.delimiter ,
-                      columns p_partkey,p_name,p_mfgr,p_brand,p_type,p_size,p_container,p_retailprice,p_comment
-                      columns.comments 
-                      columns.types int:string:string:string:string:int:string:double:string
-#### A masked pattern was here ####
-                      name default.part
-                      serialization.format 1
-                      serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                    name: default.part
-                  name: default.part
-            Truncated Path -> Alias:
-              /part [part]
-        Map 4 
-            Map Operator Tree:
-                TableScan
-                  alias: part
-                  filterExpr: (p_size is not null or (p_size is not null and p_type is not null)) (type: boolean)
-                  Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE
-                  GatherStats: false
                   Filter Operator
                     isSamplingPred: false
                     predicate: p_size is not null (type: boolean)
@@ -782,7 +736,7 @@ STAGE PLANS:
                   name: default.part
             Truncated Path -> Alias:
               /part [part]
-        Map 8 
+        Map 7 
             Map Operator Tree:
                 TableScan
                   alias: part
@@ -862,25 +816,6 @@ STAGE PLANS:
                   name: default.part
             Truncated Path -> Alias:
               /part [part]
-        Reducer 10 
-            Execution mode: vectorized, llap
-            Needs Tagging: false
-            Reduce Operator Tree:
-              Group By Operator
-                keys: KEY._col0 (type: int)
-                mode: mergepartial
-                outputColumnNames: _col0
-                Statistics: Num rows: 13 Data size: 52 Basic stats: COMPLETE Column stats: COMPLETE
-                Reduce Output Operator
-                  bucketingVersion: 2
-                  key expressions: _col0 (type: int)
-                  null sort order: z
-                  numBuckets: -1
-                  sort order: +
-                  Map-reduce partition columns: _col0 (type: int)
-                  Statistics: Num rows: 13 Data size: 52 Basic stats: COMPLETE Column stats: COMPLETE
-                  tag: 1
-                  auto parallelism: true
         Reducer 2 
             Execution mode: llap
             Needs Tagging: false
@@ -950,7 +885,7 @@ STAGE PLANS:
                       TotalFiles: 1
                       GatherStats: false
                       MultiFileSpray: false
-        Reducer 5 
+        Reducer 4 
             Needs Tagging: false
             Reduce Operator Tree:
               Group By Operator
@@ -992,7 +927,7 @@ STAGE PLANS:
                       tag: 1
                       value expressions: _col1 (type: bigint), _col2 (type: bigint)
                       auto parallelism: true
-        Reducer 6 
+        Reducer 5 
             Execution mode: vectorized, llap
             Needs Tagging: false
             Reduce Operator Tree:
@@ -1016,7 +951,7 @@ STAGE PLANS:
                     tag: 0
                     value expressions: _col0 (type: string)
                     auto parallelism: true
-        Reducer 7 
+        Reducer 6 
             Execution mode: llap
             Needs Tagging: false
             Reduce Operator Tree:
@@ -1044,6 +979,25 @@ STAGE PLANS:
                     tag: 1
                     value expressions: _col2 (type: boolean)
                     auto parallelism: true
+        Reducer 9 
+            Execution mode: vectorized, llap
+            Needs Tagging: false
+            Reduce Operator Tree:
+              Group By Operator
+                keys: KEY._col0 (type: int)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 13 Data size: 52 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  bucketingVersion: 2
+                  key expressions: _col0 (type: int)
+                  null sort order: z
+                  numBuckets: -1
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: int)
+                  Statistics: Num rows: 13 Data size: 52 Basic stats: COMPLETE Column stats: COMPLETE
+                  tag: 1
+                  auto parallelism: true
 
   Stage: Stage-0
     Fetch Operator
diff --git a/ql/src/test/results/clientpositive/llap/sharedworkresidual.q.out b/ql/src/test/results/clientpositive/llap/sharedworkresidual.q.out
index 20eb245..b35d9b8 100644
--- a/ql/src/test/results/clientpositive/llap/sharedworkresidual.q.out
+++ b/ql/src/test/results/clientpositive/llap/sharedworkresidual.q.out
@@ -40,9 +40,9 @@ POSTHOOK: query: CREATE TABLE dimension_date (
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@dimension_date
-Warning: Shuffle Join MERGEJOIN[42][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 5' is a cross product
+Warning: Shuffle Join MERGEJOIN[42][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product
 Warning: Shuffle Join MERGEJOIN[44][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
-Warning: Shuffle Join MERGEJOIN[43][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 7' is a cross product
+Warning: Shuffle Join MERGEJOIN[43][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 6' is a cross product
 Warning: Shuffle Join MERGEJOIN[45][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: explain
  with daily as (
@@ -111,12 +111,12 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Reducer 6 (CUSTOM_SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE), Reducer 8 (CUSTOM_SIMPLE_EDGE)
-        Reducer 5 <- Map 4 (CUSTOM_SIMPLE_EDGE), Map 9 (CUSTOM_SIMPLE_EDGE)
-        Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
-        Reducer 7 <- Map 4 (CUSTOM_SIMPLE_EDGE), Map 9 (CUSTOM_SIMPLE_EDGE)
-        Reducer 8 <- Reducer 7 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Reducer 5 (CUSTOM_SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE), Reducer 7 (CUSTOM_SIMPLE_EDGE)
+        Reducer 4 <- Map 1 (CUSTOM_SIMPLE_EDGE), Map 8 (CUSTOM_SIMPLE_EDGE)
+        Reducer 5 <- Reducer 4 (SIMPLE_EDGE)
+        Reducer 6 <- Map 1 (CUSTOM_SIMPLE_EDGE), Map 8 (CUSTOM_SIMPLE_EDGE)
+        Reducer 7 <- Reducer 6 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -134,17 +134,6 @@ STAGE PLANS:
                         null sort order: 
                         sort order: 
                         Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 4 
-            Map Operator Tree:
-                TableScan
-                  alias: dly
-                  filterExpr: (dateid = 20200228) (type: boolean)
-                  Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                  Filter Operator
-                    predicate: (dateid = 20200228) (type: boolean)
-                    Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: completedate (type: string)
                       outputColumnNames: _col0
@@ -161,7 +150,7 @@ STAGE PLANS:
                         value expressions: _col0 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 9 
+        Map 8 
             Map Operator Tree:
                 TableScan
                   alias: wk
@@ -220,7 +209,7 @@ STAGE PLANS:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                         serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 5 
+        Reducer 4 
             Execution mode: llap
             Reduce Operator Tree:
               Merge Join Operator
@@ -248,7 +237,7 @@ STAGE PLANS:
                       Map-reduce partition columns: _col0 (type: boolean)
                       Statistics: Num rows: 1 Data size: 373 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col1 (type: bigint)
-        Reducer 6 
+        Reducer 5 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -266,7 +255,7 @@ STAGE PLANS:
                     sort order: 
                     Statistics: Num rows: 1 Data size: 373 Basic stats: COMPLETE Column stats: NONE
                     value expressions: _col0 (type: bigint)
-        Reducer 7 
+        Reducer 6 
             Execution mode: llap
             Reduce Operator Tree:
               Merge Join Operator
@@ -294,7 +283,7 @@ STAGE PLANS:
                       Map-reduce partition columns: _col0 (type: boolean)
                       Statistics: Num rows: 1 Data size: 373 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col1 (type: bigint)
-        Reducer 8 
+        Reducer 7 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
diff --git a/ql/src/test/results/clientpositive/llap/sketches_rewrite_cume_dist_partition_by.q.out b/ql/src/test/results/clientpositive/llap/sketches_rewrite_cume_dist_partition_by.q.out
index 7873238..64235ba 100644
--- a/ql/src/test/results/clientpositive/llap/sketches_rewrite_cume_dist_partition_by.q.out
+++ b/ql/src/test/results/clientpositive/llap/sketches_rewrite_cume_dist_partition_by.q.out
@@ -81,9 +81,9 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
-        Reducer 5 <- Map 4 (SIMPLE_EDGE)
+        Reducer 4 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -102,13 +102,6 @@ STAGE PLANS:
                       Map-reduce partition columns: _col1 (type: char(1))
                       Statistics: Num rows: 26 Data size: 2163 Basic stats: COMPLETE Column stats: COMPLETE
                       value expressions: _col0 (type: int), _col2 (type: float)
-            Execution mode: vectorized, llap
-            LLAP IO: may be used (ACID table)
-        Map 4 
-            Map Operator Tree:
-                TableScan
-                  alias: sketch_input
-                  Statistics: Num rows: 26 Data size: 2059 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: category (type: char(1)), UDFToFloat(id) (type: float)
                     outputColumnNames: _col0, _col1
@@ -165,7 +158,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 5 
+        Reducer 4 
             Execution mode: llap
             Reduce Operator Tree:
               Group By Operator
diff --git a/ql/src/test/results/clientpositive/llap/sketches_rewrite_ntile_partition_by.q.out b/ql/src/test/results/clientpositive/llap/sketches_rewrite_ntile_partition_by.q.out
index c85f4cb..bea93d5 100644
--- a/ql/src/test/results/clientpositive/llap/sketches_rewrite_ntile_partition_by.q.out
+++ b/ql/src/test/results/clientpositive/llap/sketches_rewrite_ntile_partition_by.q.out
@@ -120,9 +120,9 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
-        Reducer 5 <- Map 4 (SIMPLE_EDGE)
+        Reducer 4 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -141,13 +141,6 @@ STAGE PLANS:
                       Map-reduce partition columns: _col1 (type: char(1))
                       Statistics: Num rows: 26 Data size: 2163 Basic stats: COMPLETE Column stats: COMPLETE
                       value expressions: _col0 (type: int), _col2 (type: float)
-            Execution mode: vectorized, llap
-            LLAP IO: may be used (ACID table)
-        Map 4 
-            Map Operator Tree:
-                TableScan
-                  alias: sketch_input
-                  Statistics: Num rows: 26 Data size: 2059 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: category (type: char(1)), UDFToFloat(id) (type: float)
                     outputColumnNames: _col0, _col1
@@ -204,7 +197,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 5 
+        Reducer 4 
             Execution mode: llap
             Reduce Operator Tree:
               Group By Operator
diff --git a/ql/src/test/results/clientpositive/llap/sketches_rewrite_rank_partition_by.q.out b/ql/src/test/results/clientpositive/llap/sketches_rewrite_rank_partition_by.q.out
index 769ccb6..21260a1 100644
--- a/ql/src/test/results/clientpositive/llap/sketches_rewrite_rank_partition_by.q.out
+++ b/ql/src/test/results/clientpositive/llap/sketches_rewrite_rank_partition_by.q.out
@@ -86,9 +86,9 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
-        Reducer 5 <- Map 4 (SIMPLE_EDGE)
+        Reducer 4 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -107,13 +107,6 @@ STAGE PLANS:
                       Map-reduce partition columns: _col1 (type: char(1))
                       Statistics: Num rows: 26 Data size: 2163 Basic stats: COMPLETE Column stats: COMPLETE
                       value expressions: _col0 (type: int), _col2 (type: float)
-            Execution mode: vectorized, llap
-            LLAP IO: may be used (ACID table)
-        Map 4 
-            Map Operator Tree:
-                TableScan
-                  alias: sketch_input
-                  Statistics: Num rows: 26 Data size: 2059 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: category (type: char(1)), UDFToFloat(id) (type: float)
                     outputColumnNames: _col0, _col1
@@ -170,7 +163,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 5 
+        Reducer 4 
             Execution mode: llap
             Reduce Operator Tree:
               Group By Operator
diff --git a/ql/src/test/results/clientpositive/llap/special_character_in_tabnames_1.q.out b/ql/src/test/results/clientpositive/llap/special_character_in_tabnames_1.q.out
index a19cdeb..6e18587 100644
--- a/ql/src/test/results/clientpositive/llap/special_character_in_tabnames_1.q.out
+++ b/ql/src/test/results/clientpositive/llap/special_character_in_tabnames_1.q.out
@@ -1452,14 +1452,14 @@ STAGE PLANS:
 #### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
-        Reducer 3 <- Map 4 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: b
-                  filterExpr: key is not null (type: boolean)
+                  filterExpr: (key is not null or ((value > 'val_9') and key is not null)) (type: boolean)
                   Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -1478,14 +1478,6 @@ STAGE PLANS:
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 250 Data size: 23750 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: bigint)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 4 
-            Map Operator Tree:
-                TableScan
-                  alias: a
-                  filterExpr: ((value > 'val_9') and key is not null) (type: boolean)
-                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((value > 'val_9') and key is not null) (type: boolean)
                     Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1829,18 +1821,28 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE)
-        Reducer 3 <- Map 6 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
-        Reducer 4 <- Map 7 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
-        Reducer 5 <- Reducer 4 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+        Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
+        Reducer 5 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
-                  alias: line/item
-                  filterExpr: l_partkey is not null (type: boolean)
-                  Statistics: Num rows: 100 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE
+                  alias: li
+                  filterExpr: ((l_partkey is not null and l_orderkey is not null and (l_linenumber = 1)) or l_partkey is not null or ((l_shipmode = 'AIR') and l_orderkey is not null and (l_linenumber = 1))) (type: boolean)
+                  Statistics: Num rows: 100 Data size: 1600 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: (l_partkey is not null and l_orderkey is not null and (l_linenumber = 1)) (type: boolean)
+                    Statistics: Num rows: 14 Data size: 224 Basic stats: COMPLETE Column stats: COMPLETE
+                    Reduce Output Operator
+                      key expressions: l_partkey (type: int)
+                      null sort order: z
+                      sort order: +
+                      Map-reduce partition columns: l_partkey (type: int)
+                      Statistics: Num rows: 14 Data size: 224 Basic stats: COMPLETE Column stats: COMPLETE
+                      value expressions: l_orderkey (type: int), l_suppkey (type: int)
                   Filter Operator
                     predicate: l_partkey is not null (type: boolean)
                     Statistics: Num rows: 100 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1856,32 +1858,6 @@ STAGE PLANS:
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 50 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 6 
-            Map Operator Tree:
-                TableScan
-                  alias: li
-                  filterExpr: (l_partkey is not null and l_orderkey is not null and (l_linenumber = 1)) (type: boolean)
-                  Statistics: Num rows: 100 Data size: 1600 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: (l_partkey is not null and l_orderkey is not null and (l_linenumber = 1)) (type: boolean)
-                    Statistics: Num rows: 14 Data size: 224 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: l_partkey (type: int)
-                      null sort order: z
-                      sort order: +
-                      Map-reduce partition columns: l_partkey (type: int)
-                      Statistics: Num rows: 14 Data size: 224 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: l_orderkey (type: int), l_suppkey (type: int)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 7 
-            Map Operator Tree:
-                TableScan
-                  alias: line/item
-                  filterExpr: ((l_shipmode = 'AIR') and l_orderkey is not null and (l_linenumber = 1)) (type: boolean)
-                  Statistics: Num rows: 100 Data size: 9600 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((l_shipmode = 'AIR') and l_orderkey is not null and (l_linenumber = 1)) (type: boolean)
                     Statistics: Num rows: 2 Data size: 192 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1904,20 +1880,6 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
-            Execution mode: vectorized, llap
-            Reduce Operator Tree:
-              Group By Operator
-                keys: KEY._col0 (type: int)
-                mode: mergepartial
-                outputColumnNames: _col0
-                Statistics: Num rows: 50 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE
-                Reduce Output Operator
-                  key expressions: _col0 (type: int)
-                  null sort order: z
-                  sort order: +
-                  Map-reduce partition columns: _col0 (type: int)
-                  Statistics: Num rows: 50 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE
-        Reducer 3 
             Execution mode: llap
             Reduce Operator Tree:
               Merge Join Operator
@@ -1935,7 +1897,7 @@ STAGE PLANS:
                   Map-reduce partition columns: _col1 (type: int), 1 (type: int)
                   Statistics: Num rows: 14 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col0 (type: int), _col3 (type: int)
-        Reducer 4 
+        Reducer 3 
             Execution mode: llap
             Reduce Operator Tree:
               Merge Join Operator
@@ -1956,7 +1918,7 @@ STAGE PLANS:
                     sort order: +
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col1 (type: int)
-        Reducer 5 
+        Reducer 4 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
@@ -1970,6 +1932,20 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Reducer 5 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                keys: KEY._col0 (type: int)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 50 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: _col0 (type: int)
+                  null sort order: z
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: int)
+                  Statistics: Num rows: 50 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE
 
   Stage: Stage-0
     Fetch Operator
@@ -2227,16 +2203,15 @@ STAGE PLANS:
 #### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
         Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
-        Reducer 6 <- Map 5 (SIMPLE_EDGE)
+        Reducer 5 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: p/a/r/t
-                  filterExpr: p_name is not null (type: boolean)
                   Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: p_name is not null (type: boolean)
@@ -2255,13 +2230,6 @@ STAGE PLANS:
                         Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
                         Statistics: Num rows: 13 Data size: 3835 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col2 (type: struct<count:bigint,sum:double,input:int>)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 5 
-            Map Operator Tree:
-                TableScan
-                  alias: p/a/r/t
-                  Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: p_mfgr (type: string), p_size (type: int)
                     null sort order: az
@@ -2318,7 +2286,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 6 
+        Reducer 5 
             Execution mode: llap
             Reduce Operator Tree:
               Select Operator
@@ -2408,29 +2376,16 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Reducer 6 (CUSTOM_SIMPLE_EDGE)
-        Reducer 3 <- Map 5 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
-        Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
-        Reducer 6 <- Map 5 (CUSTOM_SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+        Reducer 4 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+        Reducer 5 <- Map 1 (CUSTOM_SIMPLE_EDGE), Reducer 4 (CUSTOM_SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
-                  alias: src/_/cbo
-                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    null sort order: 
-                    sort order: 
-                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: key (type: string), value (type: string)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 5 
-            Map Operator Tree:
-                TableScan
                   alias: s1
-                  filterExpr: ((key > '2') or ((key > '2') and key is null)) (type: boolean)
                   Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key > '2') (type: boolean)
@@ -2461,6 +2416,11 @@ STAGE PLANS:
                           sort order: 
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                           value expressions: _col0 (type: bigint)
+                  Reduce Output Operator
+                    null sort order: 
+                    sort order: 
+                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                    value expressions: key (type: string), value (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
@@ -2468,24 +2428,6 @@ STAGE PLANS:
             Reduce Operator Tree:
               Merge Join Operator
                 condition map:
-                     Left Semi Join 0 to 1
-                keys:
-                  0 
-                  1 
-                outputColumnNames: _col0, _col1
-                Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                Reduce Output Operator
-                  key expressions: _col0 (type: string)
-                  null sort order: z
-                  sort order: +
-                  Map-reduce partition columns: _col0 (type: string)
-                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                  value expressions: _col1 (type: string)
-        Reducer 3 
-            Execution mode: llap
-            Reduce Operator Tree:
-              Merge Join Operator
-                condition map:
                      Left Outer Join 0 to 1
                 keys:
                   0 _col0 (type: string)
@@ -2505,7 +2447,7 @@ STAGE PLANS:
                       sort order: +
                       Statistics: Num rows: 500 Data size: 104051 Basic stats: COMPLETE Column stats: COMPLETE
                       value expressions: _col1 (type: string)
-        Reducer 4 
+        Reducer 3 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
@@ -2519,7 +2461,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 6 
+        Reducer 4 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -2542,6 +2484,24 @@ STAGE PLANS:
                         null sort order: 
                         sort order: 
                         Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+        Reducer 5 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Left Semi Join 0 to 1
+                keys:
+                  0 
+                  1 
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  null sort order: z
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col1 (type: string)
 
   Stage: Stage-0
     Fetch Operator
@@ -2590,10 +2550,10 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Reducer 6 (CUSTOM_SIMPLE_EDGE)
-        Reducer 3 <- Map 5 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Reducer 5 (CUSTOM_SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
         Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
-        Reducer 6 <- Map 5 (CUSTOM_SIMPLE_EDGE)
+        Reducer 5 <- Map 1 (CUSTOM_SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -2606,14 +2566,6 @@ STAGE PLANS:
                     sort order: 
                     Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: p_name (type: string), p_mfgr (type: string), p_size (type: int)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 5 
-            Map Operator Tree:
-                TableScan
-                  alias: p/a/r/t
-                  filterExpr: ((p_size < 10) or ((p_size < 10) and (p_name is null or p_mfgr is null))) (type: boolean)
-                  Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (p_size < 10) (type: boolean)
                     Statistics: Num rows: 5 Data size: 1115 Basic stats: COMPLETE Column stats: COMPLETE
@@ -2701,7 +2653,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 6 
+        Reducer 5 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -2772,10 +2724,10 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Reducer 6 (CUSTOM_SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Reducer 5 (CUSTOM_SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
         Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
-        Reducer 6 <- Map 5 (CUSTOM_SIMPLE_EDGE)
+        Reducer 5 <- Map 1 (CUSTOM_SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -2788,14 +2740,6 @@ STAGE PLANS:
                     sort order: 
                     Statistics: Num rows: 26 Data size: 3250 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: p_name (type: string), p_size (type: int)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 5 
-            Map Operator Tree:
-                TableScan
-                  alias: p/a/r/t
-                  filterExpr: (p_size < 10) (type: boolean)
-                  Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (p_size < 10) (type: boolean)
                     Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
@@ -2872,7 +2816,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 6 
+        Reducer 5 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -2955,12 +2899,12 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Reducer 8 (CUSTOM_SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Reducer 7 (CUSTOM_SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
         Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
-        Reducer 6 <- Map 5 (SIMPLE_EDGE)
-        Reducer 7 <- Map 5 (SIMPLE_EDGE)
-        Reducer 8 <- Reducer 7 (CUSTOM_SIMPLE_EDGE)
+        Reducer 5 <- Map 1 (SIMPLE_EDGE)
+        Reducer 6 <- Map 1 (SIMPLE_EDGE)
+        Reducer 7 <- Reducer 6 (CUSTOM_SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -2973,14 +2917,6 @@ STAGE PLANS:
                     sort order: 
                     Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: p_name (type: string), p_mfgr (type: string), p_size (type: int)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 5 
-            Map Operator Tree:
-                TableScan
-                  alias: p/a/r/t
-                  filterExpr: ((p_size < 10) or (p_size < 10)) (type: boolean)
-                  Statistics: Num rows: 26 Data size: 2652 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (p_size < 10) (type: boolean)
                     Statistics: Num rows: 5 Data size: 510 Basic stats: COMPLETE Column stats: COMPLETE
@@ -3081,7 +3017,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 6 
+        Reducer 5 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -3100,7 +3036,7 @@ STAGE PLANS:
                     sort order: ++
                     Map-reduce partition columns: _col0 (type: int), _col1 (type: string)
                     Statistics: Num rows: 2 Data size: 204 Basic stats: COMPLETE Column stats: COMPLETE
-        Reducer 7 
+        Reducer 6 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -3125,7 +3061,7 @@ STAGE PLANS:
                         sort order: 
                         Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: bigint)
-        Reducer 8 
+        Reducer 7 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -3188,18 +3124,18 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Reducer 7 (CUSTOM_SIMPLE_EDGE)
-        Reducer 3 <- Map 6 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Reducer 6 (CUSTOM_SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
         Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
         Reducer 5 <- Reducer 4 (SIMPLE_EDGE)
-        Reducer 7 <- Map 6 (CUSTOM_SIMPLE_EDGE)
+        Reducer 6 <- Map 1 (CUSTOM_SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: li
-                  filterExpr: (l_linenumber = 1) (type: boolean)
+                  filterExpr: ((l_linenumber = 1) or ((l_shipmode = 'AIR') or ((l_shipmode = 'AIR') and l_orderkey is null))) (type: boolean)
                   Statistics: Num rows: 100 Data size: 1200 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (l_linenumber = 1) (type: boolean)
@@ -3209,14 +3145,6 @@ STAGE PLANS:
                       sort order: 
                       Statistics: Num rows: 14 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
                       value expressions: l_orderkey (type: int), l_partkey (type: int)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 6 
-            Map Operator Tree:
-                TableScan
-                  alias: line/item
-                  filterExpr: ((l_shipmode = 'AIR') or ((l_shipmode = 'AIR') and l_orderkey is null)) (type: boolean)
-                  Statistics: Num rows: 100 Data size: 9200 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (l_shipmode = 'AIR') (type: boolean)
                     Statistics: Num rows: 14 Data size: 1288 Basic stats: COMPLETE Column stats: COMPLETE
@@ -3327,7 +3255,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 7 
+        Reducer 6 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
diff --git a/ql/src/test/results/clientpositive/llap/special_character_in_tabnames_quotes_1.q.out b/ql/src/test/results/clientpositive/llap/special_character_in_tabnames_quotes_1.q.out
index 9d57952..a7a9c1d 100644
--- a/ql/src/test/results/clientpositive/llap/special_character_in_tabnames_quotes_1.q.out
+++ b/ql/src/test/results/clientpositive/llap/special_character_in_tabnames_quotes_1.q.out
@@ -1624,14 +1624,14 @@ STAGE PLANS:
 #### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
-        Reducer 3 <- Map 4 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: b
-                  filterExpr: key is not null (type: boolean)
+                  filterExpr: (key is not null or ((value > 'val_9') and key is not null)) (type: boolean)
                   Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -1650,14 +1650,6 @@ STAGE PLANS:
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 250 Data size: 23750 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: bigint)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 4 
-            Map Operator Tree:
-                TableScan
-                  alias: a
-                  filterExpr: ((value > 'val_9') and key is not null) (type: boolean)
-                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((value > 'val_9') and key is not null) (type: boolean)
                     Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
@@ -2001,18 +1993,28 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE)
-        Reducer 3 <- Map 6 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
-        Reducer 4 <- Map 7 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
-        Reducer 5 <- Reducer 4 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+        Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
+        Reducer 5 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
-                  alias: line/item
-                  filterExpr: l_partkey is not null (type: boolean)
-                  Statistics: Num rows: 100 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE
+                  alias: li
+                  filterExpr: ((l_partkey is not null and l_orderkey is not null and (l_linenumber = 1)) or l_partkey is not null or ((l_shipmode = 'AIR') and l_orderkey is not null and (l_linenumber = 1))) (type: boolean)
+                  Statistics: Num rows: 100 Data size: 1600 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: (l_partkey is not null and l_orderkey is not null and (l_linenumber = 1)) (type: boolean)
+                    Statistics: Num rows: 14 Data size: 224 Basic stats: COMPLETE Column stats: COMPLETE
+                    Reduce Output Operator
+                      key expressions: l_partkey (type: int)
+                      null sort order: z
+                      sort order: +
+                      Map-reduce partition columns: l_partkey (type: int)
+                      Statistics: Num rows: 14 Data size: 224 Basic stats: COMPLETE Column stats: COMPLETE
+                      value expressions: l_orderkey (type: int), l_suppkey (type: int)
                   Filter Operator
                     predicate: l_partkey is not null (type: boolean)
                     Statistics: Num rows: 100 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE
@@ -2028,32 +2030,6 @@ STAGE PLANS:
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 50 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 6 
-            Map Operator Tree:
-                TableScan
-                  alias: li
-                  filterExpr: (l_partkey is not null and l_orderkey is not null and (l_linenumber = 1)) (type: boolean)
-                  Statistics: Num rows: 100 Data size: 1600 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: (l_partkey is not null and l_orderkey is not null and (l_linenumber = 1)) (type: boolean)
-                    Statistics: Num rows: 14 Data size: 224 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: l_partkey (type: int)
-                      null sort order: z
-                      sort order: +
-                      Map-reduce partition columns: l_partkey (type: int)
-                      Statistics: Num rows: 14 Data size: 224 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: l_orderkey (type: int), l_suppkey (type: int)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 7 
-            Map Operator Tree:
-                TableScan
-                  alias: line/item
-                  filterExpr: ((l_shipmode = 'AIR') and l_orderkey is not null and (l_linenumber = 1)) (type: boolean)
-                  Statistics: Num rows: 100 Data size: 9600 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((l_shipmode = 'AIR') and l_orderkey is not null and (l_linenumber = 1)) (type: boolean)
                     Statistics: Num rows: 2 Data size: 192 Basic stats: COMPLETE Column stats: COMPLETE
@@ -2076,20 +2052,6 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
-            Execution mode: vectorized, llap
-            Reduce Operator Tree:
-              Group By Operator
-                keys: KEY._col0 (type: int)
-                mode: mergepartial
-                outputColumnNames: _col0
-                Statistics: Num rows: 50 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE
-                Reduce Output Operator
-                  key expressions: _col0 (type: int)
-                  null sort order: z
-                  sort order: +
-                  Map-reduce partition columns: _col0 (type: int)
-                  Statistics: Num rows: 50 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE
-        Reducer 3 
             Execution mode: llap
             Reduce Operator Tree:
               Merge Join Operator
@@ -2107,7 +2069,7 @@ STAGE PLANS:
                   Map-reduce partition columns: _col1 (type: int), 1 (type: int)
                   Statistics: Num rows: 14 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col0 (type: int), _col3 (type: int)
-        Reducer 4 
+        Reducer 3 
             Execution mode: llap
             Reduce Operator Tree:
               Merge Join Operator
@@ -2128,7 +2090,7 @@ STAGE PLANS:
                     sort order: +
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col1 (type: int)
-        Reducer 5 
+        Reducer 4 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
@@ -2142,6 +2104,20 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Reducer 5 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                keys: KEY._col0 (type: int)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 50 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: _col0 (type: int)
+                  null sort order: z
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: int)
+                  Statistics: Num rows: 50 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE
 
   Stage: Stage-0
     Fetch Operator
@@ -2399,16 +2375,15 @@ STAGE PLANS:
 #### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
         Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
-        Reducer 6 <- Map 5 (SIMPLE_EDGE)
+        Reducer 5 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: p/a/r/t
-                  filterExpr: p_name is not null (type: boolean)
                   Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: p_name is not null (type: boolean)
@@ -2427,13 +2402,6 @@ STAGE PLANS:
                         Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
                         Statistics: Num rows: 13 Data size: 3835 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col2 (type: struct<count:bigint,sum:double,input:int>)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 5 
-            Map Operator Tree:
-                TableScan
-                  alias: p/a/r/t
-                  Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: p_mfgr (type: string), p_size (type: int)
                     null sort order: az
@@ -2490,7 +2458,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 6 
+        Reducer 5 
             Execution mode: llap
             Reduce Operator Tree:
               Select Operator
@@ -2580,29 +2548,16 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Reducer 6 (CUSTOM_SIMPLE_EDGE)
-        Reducer 3 <- Map 5 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
-        Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
-        Reducer 6 <- Map 5 (CUSTOM_SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+        Reducer 4 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+        Reducer 5 <- Map 1 (CUSTOM_SIMPLE_EDGE), Reducer 4 (CUSTOM_SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
-                  alias: src/_/cbo
-                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    null sort order: 
-                    sort order: 
-                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: key (type: string), value (type: string)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 5 
-            Map Operator Tree:
-                TableScan
                   alias: s1
-                  filterExpr: ((key > '2') or ((key > '2') and key is null)) (type: boolean)
                   Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key > '2') (type: boolean)
@@ -2633,6 +2588,11 @@ STAGE PLANS:
                           sort order: 
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                           value expressions: _col0 (type: bigint)
+                  Reduce Output Operator
+                    null sort order: 
+                    sort order: 
+                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                    value expressions: key (type: string), value (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
@@ -2640,24 +2600,6 @@ STAGE PLANS:
             Reduce Operator Tree:
               Merge Join Operator
                 condition map:
-                     Left Semi Join 0 to 1
-                keys:
-                  0 
-                  1 
-                outputColumnNames: _col0, _col1
-                Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                Reduce Output Operator
-                  key expressions: _col0 (type: string)
-                  null sort order: z
-                  sort order: +
-                  Map-reduce partition columns: _col0 (type: string)
-                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                  value expressions: _col1 (type: string)
-        Reducer 3 
-            Execution mode: llap
-            Reduce Operator Tree:
-              Merge Join Operator
-                condition map:
                      Left Outer Join 0 to 1
                 keys:
                   0 _col0 (type: string)
@@ -2677,7 +2619,7 @@ STAGE PLANS:
                       sort order: +
                       Statistics: Num rows: 500 Data size: 104051 Basic stats: COMPLETE Column stats: COMPLETE
                       value expressions: _col1 (type: string)
-        Reducer 4 
+        Reducer 3 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
@@ -2691,7 +2633,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 6 
+        Reducer 4 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -2714,6 +2656,24 @@ STAGE PLANS:
                         null sort order: 
                         sort order: 
                         Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+        Reducer 5 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Left Semi Join 0 to 1
+                keys:
+                  0 
+                  1 
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  null sort order: z
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col1 (type: string)
 
   Stage: Stage-0
     Fetch Operator
@@ -2762,10 +2722,10 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Reducer 6 (CUSTOM_SIMPLE_EDGE)
-        Reducer 3 <- Map 5 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Reducer 5 (CUSTOM_SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
         Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
-        Reducer 6 <- Map 5 (CUSTOM_SIMPLE_EDGE)
+        Reducer 5 <- Map 1 (CUSTOM_SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -2778,14 +2738,6 @@ STAGE PLANS:
                     sort order: 
                     Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: p_name (type: string), p_mfgr (type: string), p_size (type: int)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 5 
-            Map Operator Tree:
-                TableScan
-                  alias: p/a/r/t
-                  filterExpr: ((p_size < 10) or ((p_size < 10) and (p_name is null or p_mfgr is null))) (type: boolean)
-                  Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (p_size < 10) (type: boolean)
                     Statistics: Num rows: 5 Data size: 1115 Basic stats: COMPLETE Column stats: COMPLETE
@@ -2873,7 +2825,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 6 
+        Reducer 5 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -2944,10 +2896,10 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Reducer 6 (CUSTOM_SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Reducer 5 (CUSTOM_SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
         Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
-        Reducer 6 <- Map 5 (CUSTOM_SIMPLE_EDGE)
+        Reducer 5 <- Map 1 (CUSTOM_SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -2960,14 +2912,6 @@ STAGE PLANS:
                     sort order: 
                     Statistics: Num rows: 26 Data size: 3250 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: p_name (type: string), p_size (type: int)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 5 
-            Map Operator Tree:
-                TableScan
-                  alias: p/a/r/t
-                  filterExpr: (p_size < 10) (type: boolean)
-                  Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (p_size < 10) (type: boolean)
                     Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
@@ -3044,7 +2988,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 6 
+        Reducer 5 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -3127,12 +3071,12 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Reducer 8 (CUSTOM_SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Reducer 7 (CUSTOM_SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
         Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
-        Reducer 6 <- Map 5 (SIMPLE_EDGE)
-        Reducer 7 <- Map 5 (SIMPLE_EDGE)
-        Reducer 8 <- Reducer 7 (CUSTOM_SIMPLE_EDGE)
+        Reducer 5 <- Map 1 (SIMPLE_EDGE)
+        Reducer 6 <- Map 1 (SIMPLE_EDGE)
+        Reducer 7 <- Reducer 6 (CUSTOM_SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -3145,14 +3089,6 @@ STAGE PLANS:
                     sort order: 
                     Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: p_name (type: string), p_mfgr (type: string), p_size (type: int)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 5 
-            Map Operator Tree:
-                TableScan
-                  alias: p/a/r/t
-                  filterExpr: ((p_size < 10) or (p_size < 10)) (type: boolean)
-                  Statistics: Num rows: 26 Data size: 2652 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (p_size < 10) (type: boolean)
                     Statistics: Num rows: 5 Data size: 510 Basic stats: COMPLETE Column stats: COMPLETE
@@ -3253,7 +3189,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 6 
+        Reducer 5 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -3272,7 +3208,7 @@ STAGE PLANS:
                     sort order: ++
                     Map-reduce partition columns: _col0 (type: int), _col1 (type: string)
                     Statistics: Num rows: 2 Data size: 204 Basic stats: COMPLETE Column stats: COMPLETE
-        Reducer 7 
+        Reducer 6 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -3297,7 +3233,7 @@ STAGE PLANS:
                         sort order: 
                         Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: bigint)
-        Reducer 8 
+        Reducer 7 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -3360,18 +3296,18 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Reducer 7 (CUSTOM_SIMPLE_EDGE)
-        Reducer 3 <- Map 6 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Reducer 6 (CUSTOM_SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
         Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
         Reducer 5 <- Reducer 4 (SIMPLE_EDGE)
-        Reducer 7 <- Map 6 (CUSTOM_SIMPLE_EDGE)
+        Reducer 6 <- Map 1 (CUSTOM_SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: li
-                  filterExpr: (l_linenumber = 1) (type: boolean)
+                  filterExpr: ((l_linenumber = 1) or ((l_shipmode = 'AIR') or ((l_shipmode = 'AIR') and l_orderkey is null))) (type: boolean)
                   Statistics: Num rows: 100 Data size: 1200 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (l_linenumber = 1) (type: boolean)
@@ -3381,14 +3317,6 @@ STAGE PLANS:
                       sort order: 
                       Statistics: Num rows: 14 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
                       value expressions: l_orderkey (type: int), l_partkey (type: int)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 6 
-            Map Operator Tree:
-                TableScan
-                  alias: line/item
-                  filterExpr: ((l_shipmode = 'AIR') or ((l_shipmode = 'AIR') and l_orderkey is null)) (type: boolean)
-                  Statistics: Num rows: 100 Data size: 9200 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (l_shipmode = 'AIR') (type: boolean)
                     Statistics: Num rows: 14 Data size: 1288 Basic stats: COMPLETE Column stats: COMPLETE
@@ -3499,7 +3427,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 7 
+        Reducer 6 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
diff --git a/ql/src/test/results/clientpositive/llap/subquery_ALL.q.out b/ql/src/test/results/clientpositive/llap/subquery_ALL.q.out
index d05e7c7..3f548d7 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_ALL.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_ALL.q.out
@@ -469,7 +469,7 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@part
 #### A masked pattern was here ####
 26
-Warning: Shuffle Join MERGEJOIN[52][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 9' is a cross product
+Warning: Shuffle Join MERGEJOIN[52][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 8' is a cross product
 Warning: Shuffle Join MERGEJOIN[53][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: explain select count(*) from part where p_partkey >= ALL (select p_partkey from part)
 	AND p_size <> ALL (select p_size from part group by p_size)
@@ -490,14 +490,14 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 11 <- Map 10 (CUSTOM_SIMPLE_EDGE)
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (XPROD_EDGE), Reducer 9 (XPROD_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (XPROD_EDGE), Reducer 8 (XPROD_EDGE)
         Reducer 4 <- Reducer 3 (CUSTOM_SIMPLE_EDGE)
-        Reducer 6 <- Map 5 (SIMPLE_EDGE)
-        Reducer 7 <- Map 5 (SIMPLE_EDGE)
-        Reducer 8 <- Reducer 7 (CUSTOM_SIMPLE_EDGE)
-        Reducer 9 <- Reducer 11 (XPROD_EDGE), Reducer 8 (XPROD_EDGE)
+        Reducer 5 <- Map 1 (SIMPLE_EDGE)
+        Reducer 6 <- Map 1 (SIMPLE_EDGE)
+        Reducer 7 <- Reducer 6 (CUSTOM_SIMPLE_EDGE)
+        Reducer 8 <- Reducer 7 (XPROD_EDGE), Reducer 9 (XPROD_EDGE)
+        Reducer 9 <- Map 1 (CUSTOM_SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -516,35 +516,6 @@ STAGE PLANS:
                       Map-reduce partition columns: _col1 (type: int)
                       Statistics: Num rows: 26 Data size: 208 Basic stats: COMPLETE Column stats: COMPLETE
                       value expressions: _col0 (type: int)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 10 
-            Map Operator Tree:
-                TableScan
-                  alias: part
-                  Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: p_partkey (type: int)
-                    outputColumnNames: p_partkey
-                    Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
-                    Group By Operator
-                      aggregations: max(p_partkey), count(), count(p_partkey)
-                      minReductionHashAggr: 0.96153843
-                      mode: hash
-                      outputColumnNames: _col0, _col1, _col2
-                      Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        null sort order: 
-                        sort order: 
-                        Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: bigint)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 5 
-            Map Operator Tree:
-                TableScan
-                  alias: part
-                  Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: p_size is not null (type: boolean)
                     Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
@@ -576,21 +547,23 @@ STAGE PLANS:
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 13 Data size: 52 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: p_partkey (type: int)
+                    outputColumnNames: p_partkey
+                    Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
+                    Group By Operator
+                      aggregations: max(p_partkey), count(), count(p_partkey)
+                      minReductionHashAggr: 0.96153843
+                      mode: hash
+                      outputColumnNames: _col0, _col1, _col2
+                      Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        null sort order: 
+                        sort order: 
+                        Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: bigint)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Reducer 11 
-            Execution mode: vectorized, llap
-            Reduce Operator Tree:
-              Group By Operator
-                aggregations: max(VALUE._col0), count(VALUE._col1), count(VALUE._col2)
-                mode: mergepartial
-                outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
-                Reduce Output Operator
-                  null sort order: 
-                  sort order: 
-                  Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
-                  value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: bigint)
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
@@ -654,7 +627,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 6 
+        Reducer 5 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -673,7 +646,7 @@ STAGE PLANS:
                     Map-reduce partition columns: _col0 (type: int)
                     Statistics: Num rows: 13 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col1 (type: boolean)
-        Reducer 7 
+        Reducer 6 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -692,7 +665,7 @@ STAGE PLANS:
                     sort order: 
                     Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col0 (type: bigint), _col1 (type: bigint)
-        Reducer 8 
+        Reducer 7 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -705,7 +678,7 @@ STAGE PLANS:
                   sort order: 
                   Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col0 (type: bigint), _col1 (type: bigint)
-        Reducer 9 
+        Reducer 8 
             Execution mode: llap
             Reduce Operator Tree:
               Merge Join Operator
@@ -721,6 +694,19 @@ STAGE PLANS:
                   sort order: 
                   Statistics: Num rows: 1 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: bigint)
+        Reducer 9 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: max(VALUE._col0), count(VALUE._col1), count(VALUE._col2)
+                mode: mergepartial
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  null sort order: 
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: bigint)
 
   Stage: Stage-0
     Fetch Operator
@@ -741,7 +727,7 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@part
 #### A masked pattern was here ####
 1
-Warning: Shuffle Join MERGEJOIN[32][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 5' is a cross product
+Warning: Shuffle Join MERGEJOIN[32][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product
 Warning: Shuffle Join MERGEJOIN[33][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
 PREHOOK: query: explain cbo select count(*) from part where p_partkey 
 	>= ALL (select p_partkey from part where p_size >= ALL(select p_size from part_null_n0 group by p_size))
@@ -771,7 +757,7 @@ HiveAggregate(group=[{}], agg#0=[count()])
                 HiveAggregate(group=[{5}])
                   HiveTableScan(table=[[default, part_null_n0]], table:alias=[part_null_n0])
 
-Warning: Shuffle Join MERGEJOIN[32][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 5' is a cross product
+Warning: Shuffle Join MERGEJOIN[32][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product
 Warning: Shuffle Join MERGEJOIN[33][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
 PREHOOK: query: select count(*) from part where p_partkey 
 	>= ALL (select p_partkey from part where p_size >= ALL(select p_size from part_null_n0 group by p_size))
diff --git a/ql/src/test/results/clientpositive/llap/subquery_ANY.q.out b/ql/src/test/results/clientpositive/llap/subquery_ANY.q.out
index 09b26e2..2797890 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_ANY.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_ANY.q.out
@@ -376,7 +376,7 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@part
 #### A masked pattern was here ####
 26
-Warning: Shuffle Join MERGEJOIN[43][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[43][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 5' is a cross product
 PREHOOK: query: explain cbo select count(*) from part where p_partkey >= ANY (select p_partkey from part) 
 	AND p_size = ANY (select p_size from part group by p_size)
 PREHOOK: type: QUERY
@@ -402,7 +402,7 @@ HiveAggregate(group=[{}], agg#0=[count()])
         HiveAggregate(group=[{}], m=[MIN($0)], c=[COUNT()], d=[COUNT($0)])
           HiveTableScan(table=[[default, part]], table:alias=[part])
 
-Warning: Shuffle Join MERGEJOIN[43][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[43][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 5' is a cross product
 PREHOOK: query: select count(*) from part where p_partkey >= ANY (select p_partkey from part) 
 	AND p_size = ANY (select p_size from part group by p_size)
 PREHOOK: type: QUERY
@@ -414,7 +414,7 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@part
 #### A masked pattern was here ####
 26
-Warning: Shuffle Join MERGEJOIN[32][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 5' is a cross product
+Warning: Shuffle Join MERGEJOIN[32][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product
 Warning: Shuffle Join MERGEJOIN[33][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
 PREHOOK: query: explain cbo select count(*) from part where p_partkey 
 	>= ANY (select p_partkey from part where p_size >= ANY(select p_size from part_null_n0 group by p_size))
@@ -444,7 +444,7 @@ HiveAggregate(group=[{}], agg#0=[count()])
                 HiveAggregate(group=[{5}])
                   HiveTableScan(table=[[default, part_null_n0]], table:alias=[part_null_n0])
 
-Warning: Shuffle Join MERGEJOIN[32][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 5' is a cross product
+Warning: Shuffle Join MERGEJOIN[32][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product
 Warning: Shuffle Join MERGEJOIN[33][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
 PREHOOK: query: select count(*) from part where p_partkey 
 	>= ANY (select p_partkey from part where p_size >= ANY(select p_size from part_null_n0 group by p_size))
@@ -621,8 +621,8 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-        Reducer 4 <- Map 3 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -641,14 +641,6 @@ STAGE PLANS:
                       Map-reduce partition columns: _col4 (type: string)
                       Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
                       value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 3 
-            Map Operator Tree:
-                TableScan
-                  alias: p
-                  filterExpr: p_type is not null (type: boolean)
-                  Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: p_type is not null (type: boolean)
                     Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE
@@ -693,7 +685,7 @@ STAGE PLANS:
                           input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                           output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                           serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 4 
+        Reducer 3 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -746,8 +738,8 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-        Reducer 4 <- Map 3 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -766,14 +758,6 @@ STAGE PLANS:
                       Map-reduce partition columns: _col1 (type: string)
                       Statistics: Num rows: 26 Data size: 3250 Basic stats: COMPLETE Column stats: COMPLETE
                       value expressions: _col0 (type: int)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 3 
-            Map Operator Tree:
-                TableScan
-                  alias: pp
-                  filterExpr: p_type is not null (type: boolean)
-                  Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: p_type is not null (type: boolean)
                     Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE
@@ -815,7 +799,7 @@ STAGE PLANS:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                         serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 4 
+        Reducer 3 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -1326,8 +1310,8 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-        Reducer 4 <- Map 3 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -1346,14 +1330,6 @@ STAGE PLANS:
                       Map-reduce partition columns: _col1 (type: string)
                       Statistics: Num rows: 26 Data size: 3250 Basic stats: COMPLETE Column stats: COMPLETE
                       value expressions: _col0 (type: int)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 3 
-            Map Operator Tree:
-                TableScan
-                  alias: pp
-                  filterExpr: p_type is not null (type: boolean)
-                  Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: p_type is not null (type: boolean)
                     Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1395,7 +1371,7 @@ STAGE PLANS:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                         serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 4 
+        Reducer 3 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -1488,8 +1464,8 @@ STAGE PLANS:
 #### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-        Reducer 4 <- Map 3 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
-        Reducer 6 <- Map 5 (SIMPLE_EDGE)
+        Reducer 4 <- Map 3 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
+        Reducer 5 <- Map 3 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -1518,7 +1494,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: part
-                  filterExpr: (p_type is not null and p_name is not null) (type: boolean)
+                  filterExpr: ((p_type is not null and p_name is not null) or p_type is not null) (type: boolean)
                   Statistics: Num rows: 26 Data size: 5954 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (p_type is not null and p_name is not null) (type: boolean)
@@ -1534,14 +1510,6 @@ STAGE PLANS:
                         Map-reduce partition columns: _col1 (type: string)
                         Statistics: Num rows: 26 Data size: 5954 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: string), _col2 (type: int)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 5 
-            Map Operator Tree:
-                TableScan
-                  alias: pp
-                  filterExpr: p_type is not null (type: boolean)
-                  Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: p_type is not null (type: boolean)
                     Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1609,7 +1577,7 @@ STAGE PLANS:
                         sort order: ++
                         Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
                         Statistics: Num rows: 3 Data size: 675 Basic stats: COMPLETE Column stats: COMPLETE
-        Reducer 6 
+        Reducer 5 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -1696,8 +1664,8 @@ STAGE PLANS:
 #### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
-        Reducer 6 <- Map 5 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
+        Reducer 5 <- Map 4 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -1726,7 +1694,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: part
-                  filterExpr: (p_type is not null and p_name is not null) (type: boolean)
+                  filterExpr: ((p_type is not null and p_name is not null) or p_type is not null) (type: boolean)
                   Statistics: Num rows: 26 Data size: 5850 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (p_type is not null and p_name is not null) (type: boolean)
@@ -1747,14 +1715,6 @@ STAGE PLANS:
                           sort order: ++
                           Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
                           Statistics: Num rows: 13 Data size: 2925 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 5 
-            Map Operator Tree:
-                TableScan
-                  alias: pp
-                  filterExpr: p_type is not null (type: boolean)
-                  Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: p_type is not null (type: boolean)
                     Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1817,7 +1777,7 @@ STAGE PLANS:
                           input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                           output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                           serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 6 
+        Reducer 5 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
diff --git a/ql/src/test/results/clientpositive/llap/subquery_exists.q.out b/ql/src/test/results/clientpositive/llap/subquery_exists.q.out
index a9e2f83..22013c4 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_exists.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_exists.q.out
@@ -354,8 +354,8 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 4 (XPROD_EDGE)
-        Reducer 4 <- Map 3 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 3 (XPROD_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -372,14 +372,6 @@ STAGE PLANS:
                       sort order: 
                       Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                       value expressions: _col0 (type: string), _col1 (type: string)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 3 
-            Map Operator Tree:
-                TableScan
-                  alias: a
-                  filterExpr: (value > 'val_9') (type: boolean)
-                  Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (value > 'val_9') (type: boolean)
                     Statistics: Num rows: 166 Data size: 15106 Basic stats: COMPLETE Column stats: COMPLETE
@@ -417,7 +409,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 4 
+        Reducer 3 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
diff --git a/ql/src/test/results/clientpositive/llap/subquery_exists_having.q.out b/ql/src/test/results/clientpositive/llap/subquery_exists_having.q.out
index dc6b1af..9e5e354 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_exists_having.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_exists_having.q.out
@@ -32,14 +32,14 @@ STAGE PLANS:
 #### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
-        Reducer 3 <- Map 4 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: b
-                  filterExpr: key is not null (type: boolean)
+                  filterExpr: (key is not null or ((value > 'val_9') and key is not null)) (type: boolean)
                   Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -58,14 +58,6 @@ STAGE PLANS:
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 250 Data size: 23750 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: bigint)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 4 
-            Map Operator Tree:
-                TableScan
-                  alias: a
-                  filterExpr: ((value > 'val_9') and key is not null) (type: boolean)
-                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((value > 'val_9') and key is not null) (type: boolean)
                     Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
@@ -190,14 +182,14 @@ STAGE PLANS:
 #### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
-        Reducer 3 <- Map 4 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: b
-                  filterExpr: key is not null (type: boolean)
+                  filterExpr: (key is not null or ((value > 'val_9') and key is not null)) (type: boolean)
                   Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -216,14 +208,6 @@ STAGE PLANS:
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 250 Data size: 23750 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: bigint)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 4 
-            Map Operator Tree:
-                TableScan
-                  alias: a
-                  filterExpr: ((value > 'val_9') and key is not null) (type: boolean)
-                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((value > 'val_9') and key is not null) (type: boolean)
                     Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/subquery_in.q.out b/ql/src/test/results/clientpositive/llap/subquery_in.q.out
index e9b6be9..8ec86d0 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_in.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_in.q.out
@@ -290,16 +290,15 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
-        Reducer 4 <- Map 3 (SIMPLE_EDGE)
-        Reducer 5 <- Reducer 4 (CUSTOM_SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE)
+        Reducer 4 <- Reducer 3 (CUSTOM_SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: part
-                  filterExpr: UDFToDouble(p_size) is not null (type: boolean)
                   Statistics: Num rows: 26 Data size: 3250 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: UDFToDouble(p_size) is not null (type: boolean)
@@ -315,13 +314,6 @@ STAGE PLANS:
                         Map-reduce partition columns: UDFToDouble(_col1) (type: double)
                         Statistics: Num rows: 26 Data size: 3250 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: string), _col1 (type: int)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 3 
-            Map Operator Tree:
-                TableScan
-                  alias: part
-                  Statistics: Num rows: 26 Data size: 2652 Basic stats: COMPLETE Column stats: COMPLETE
                   Top N Key Operator
                     sort order: ++
                     keys: p_mfgr (type: string), p_size (type: int)
@@ -355,7 +347,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 4 
+        Reducer 3 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
@@ -401,7 +393,7 @@ STAGE PLANS:
                           sort order: 
                           Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
                           value expressions: _col0 (type: bigint), _col1 (type: bigint)
-        Reducer 5 
+        Reducer 4 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -480,16 +472,16 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
-        Reducer 4 <- Map 3 (SIMPLE_EDGE)
-        Reducer 5 <- Reducer 4 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE)
+        Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: b
-                  filterExpr: (p_mfgr is not null and p_size is not null) (type: boolean)
+                  filterExpr: ((p_mfgr is not null and p_size is not null) or p_mfgr is not null) (type: boolean)
                   Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (p_mfgr is not null and p_size is not null) (type: boolean)
@@ -505,14 +497,6 @@ STAGE PLANS:
                         Map-reduce partition columns: _col1 (type: string), _col2 (type: int)
                         Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: string)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 3 
-            Map Operator Tree:
-                TableScan
-                  alias: part
-                  filterExpr: p_mfgr is not null (type: boolean)
-                  Statistics: Num rows: 26 Data size: 2652 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: p_mfgr is not null (type: boolean)
                     Statistics: Num rows: 26 Data size: 2652 Basic stats: COMPLETE Column stats: COMPLETE
@@ -553,7 +537,7 @@ STAGE PLANS:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                         serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 4 
+        Reducer 3 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
@@ -602,7 +586,7 @@ STAGE PLANS:
                           Map-reduce partition columns: _col0 (type: string)
                           Statistics: Num rows: 4 Data size: 408 Basic stats: COMPLETE Column stats: COMPLETE
                           value expressions: _col1 (type: int)
-        Reducer 5 
+        Reducer 4 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -826,15 +810,15 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-        Reducer 4 <- Map 3 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: b
-                  filterExpr: key is not null (type: boolean)
+                  filterExpr: (key is not null or (key > '9')) (type: boolean)
                   Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -850,14 +834,6 @@ STAGE PLANS:
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: string)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 3 
-            Map Operator Tree:
-                TableScan
-                  alias: a
-                  filterExpr: (key > '9') (type: boolean)
-                  Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key > '9') (type: boolean)
                     Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE
@@ -898,7 +874,7 @@ STAGE PLANS:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                         serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 4 
+        Reducer 3 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -1120,16 +1096,16 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
-        Reducer 3 <- Map 6 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
-        Reducer 5 <- Map 4 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+        Reducer 4 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: li
-                  filterExpr: ((l_linenumber = 1) and l_partkey is not null and l_orderkey is not null) (type: boolean)
+                  filterExpr: (((l_linenumber = 1) and l_partkey is not null and l_orderkey is not null) or l_partkey is not null or ((l_shipmode = 'AIR') and l_orderkey is not null)) (type: boolean)
                   Statistics: Num rows: 100 Data size: 1600 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((l_linenumber = 1) and l_partkey is not null and l_orderkey is not null) (type: boolean)
@@ -1145,14 +1121,6 @@ STAGE PLANS:
                         Map-reduce partition columns: _col1 (type: int)
                         Statistics: Num rows: 14 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int), _col2 (type: int)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 4 
-            Map Operator Tree:
-                TableScan
-                  alias: lineitem
-                  filterExpr: l_partkey is not null (type: boolean)
-                  Statistics: Num rows: 100 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: l_partkey is not null (type: boolean)
                     Statistics: Num rows: 100 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1168,14 +1136,6 @@ STAGE PLANS:
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 50 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 6 
-            Map Operator Tree:
-                TableScan
-                  alias: lineitem
-                  filterExpr: ((l_shipmode = 'AIR') and l_orderkey is not null) (type: boolean)
-                  Statistics: Num rows: 100 Data size: 9200 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((l_shipmode = 'AIR') and l_orderkey is not null) (type: boolean)
                     Statistics: Num rows: 14 Data size: 1288 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1237,7 +1197,7 @@ STAGE PLANS:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                         serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 5 
+        Reducer 4 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -1317,17 +1277,17 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 6 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
         Reducer 4 <- Reducer 3 (CUSTOM_SIMPLE_EDGE)
-        Reducer 7 <- Map 6 (SIMPLE_EDGE)
+        Reducer 5 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: lineitem
-                  filterExpr: (l_partkey is not null and l_quantity is not null) (type: boolean)
+                  filterExpr: ((l_partkey is not null and l_quantity is not null) or l_partkey is not null) (type: boolean)
                   Statistics: Num rows: 100 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (l_partkey is not null and l_quantity is not null) (type: boolean)
@@ -1343,9 +1303,26 @@ STAGE PLANS:
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 100 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: double), _col2 (type: double)
+                  Filter Operator
+                    predicate: l_partkey is not null (type: boolean)
+                    Statistics: Num rows: 100 Data size: 1200 Basic stats: COMPLETE Column stats: COMPLETE
+                    Group By Operator
+                      aggregations: sum(l_quantity), count(l_quantity)
+                      keys: l_partkey (type: int)
+                      minReductionHashAggr: 0.5
+                      mode: hash
+                      outputColumnNames: _col0, _col1, _col2
+                      Statistics: Num rows: 50 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 50 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: double), _col2 (type: bigint)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 5 
+        Map 6 
             Map Operator Tree:
                 TableScan
                   alias: part
@@ -1366,31 +1343,6 @@ STAGE PLANS:
                         Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 6 
-            Map Operator Tree:
-                TableScan
-                  alias: lineitem
-                  filterExpr: l_partkey is not null (type: boolean)
-                  Statistics: Num rows: 100 Data size: 1200 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: l_partkey is not null (type: boolean)
-                    Statistics: Num rows: 100 Data size: 1200 Basic stats: COMPLETE Column stats: COMPLETE
-                    Group By Operator
-                      aggregations: sum(l_quantity), count(l_quantity)
-                      keys: l_partkey (type: int)
-                      minReductionHashAggr: 0.5
-                      mode: hash
-                      outputColumnNames: _col0, _col1, _col2
-                      Statistics: Num rows: 50 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: int)
-                        null sort order: z
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 50 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions: _col1 (type: double), _col2 (type: bigint)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
@@ -1446,7 +1398,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 7 
+        Reducer 5 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -1509,15 +1461,15 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-        Reducer 4 <- Map 3 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: part
-                  filterExpr: ((p_size <> 340) and (p_brand <> 'Brand#14') and p_type is not null) (type: boolean)
+                  filterExpr: (((p_size <> 340) and (p_brand <> 'Brand#14') and p_type is not null) or ((p_size <> 340) and p_type is not null)) (type: boolean)
                   Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((p_size <> 340) and (p_brand <> 'Brand#14') and p_type is not null) (type: boolean)
@@ -1533,14 +1485,6 @@ STAGE PLANS:
                         Map-reduce partition columns: _col4 (type: string), _col5 (type: int)
                         Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col6 (type: string), _col7 (type: double), _col8 (type: string)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 3 
-            Map Operator Tree:
-                TableScan
-                  alias: p
-                  filterExpr: ((p_size <> 340) and p_type is not null) (type: boolean)
-                  Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((p_size <> 340) and p_type is not null) (type: boolean)
                     Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1576,7 +1520,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 4 
+        Reducer 3 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -1652,15 +1596,14 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-        Reducer 4 <- Map 3 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: part
-                  filterExpr: p_size is not null (type: boolean)
                   Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: p_size is not null (type: boolean)
@@ -1676,13 +1619,6 @@ STAGE PLANS:
                         Map-reduce partition columns: _col9 (type: int)
                         Statistics: Num rows: 26 Data size: 16198 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 3 
-            Map Operator Tree:
-                TableScan
-                  alias: part
-                  Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: p_type (type: string), p_size (type: int)
                     outputColumnNames: p_type, p_size
@@ -1721,7 +1657,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 4 
+        Reducer 3 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -1795,15 +1731,14 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-        Reducer 4 <- Map 3 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: part
-                  filterExpr: (p_partkey is not null and p_size is not null) (type: boolean)
                   Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (p_partkey is not null and p_size is not null) (type: boolean)
@@ -1819,13 +1754,6 @@ STAGE PLANS:
                         Map-reduce partition columns: _col9 (type: int)
                         Statistics: Num rows: 26 Data size: 16198 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 3 
-            Map Operator Tree:
-                TableScan
-                  alias: part
-                  Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: p_partkey (type: int), p_type (type: string)
                     outputColumnNames: p_partkey, p_type
@@ -1864,7 +1792,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 4 
+        Reducer 3 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -2050,15 +1978,14 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-        Reducer 4 <- Map 3 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: part
-                  filterExpr: p_retailprice is not null (type: boolean)
                   Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: p_retailprice is not null (type: boolean)
@@ -2074,13 +2001,6 @@ STAGE PLANS:
                         Map-reduce partition columns: _col9 (type: bigint)
                         Statistics: Num rows: 26 Data size: 16302 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 3 
-            Map Operator Tree:
-                TableScan
-                  alias: part
-                  Statistics: Num rows: 26 Data size: 2912 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: p_type (type: string), p_retailprice (type: double)
                     outputColumnNames: p_type, p_retailprice
@@ -2119,7 +2039,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 4 
+        Reducer 3 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -2568,16 +2488,16 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-        Reducer 4 <- Map 3 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
-        Reducer 6 <- Map 5 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
+        Reducer 4 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: part
-                  filterExpr: (p_size is not null and p_type is not null) (type: boolean)
+                  filterExpr: ((p_size is not null and p_type is not null) or p_size is not null) (type: boolean)
                   Statistics: Num rows: 26 Data size: 5954 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (p_size is not null and p_type is not null) (type: boolean)
@@ -2593,17 +2513,6 @@ STAGE PLANS:
                         Map-reduce partition columns: _col1 (type: string), _col2 (type: int)
                         Statistics: Num rows: 26 Data size: 5954 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: string)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 3 
-            Map Operator Tree:
-                TableScan
-                  alias: part
-                  filterExpr: (p_size is not null and p_type is not null) (type: boolean)
-                  Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: (p_size is not null and p_type is not null) (type: boolean)
-                    Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: p_type (type: string), (p_size + 1) (type: int)
                       outputColumnNames: _col0, _col1
@@ -2615,14 +2524,6 @@ STAGE PLANS:
                         Map-reduce partition columns: _col1 (type: int)
                         Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: string)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 5 
-            Map Operator Tree:
-                TableScan
-                  alias: part
-                  filterExpr: p_size is not null (type: boolean)
-                  Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: p_size is not null (type: boolean)
                     Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
@@ -2662,7 +2563,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 4 
+        Reducer 3 
             Execution mode: llap
             Reduce Operator Tree:
               Merge Join Operator
@@ -2689,7 +2590,7 @@ STAGE PLANS:
                       sort order: ++
                       Map-reduce partition columns: _col0 (type: string), _col1 (type: int)
                       Statistics: Num rows: 8 Data size: 864 Basic stats: COMPLETE Column stats: COMPLETE
-        Reducer 6 
+        Reducer 4 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -2761,17 +2662,17 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 6 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
-        Reducer 4 <- Reducer 3 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE)
-        Reducer 7 <- Map 6 (SIMPLE_EDGE)
+        Reducer 4 <- Reducer 3 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
+        Reducer 5 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: src
-                  filterExpr: value is not null (type: boolean)
+                  filterExpr: (value is not null or (key = '90')) (type: boolean)
                   Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: value is not null (type: boolean)
@@ -2787,9 +2688,28 @@ STAGE PLANS:
                         Map-reduce partition columns: _col1 (type: string)
                         Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: string)
+                  Filter Operator
+                    predicate: (key = '90') (type: boolean)
+                    Statistics: Num rows: 2 Data size: 174 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      Statistics: Num rows: 2 Data size: 174 Basic stats: COMPLETE Column stats: COMPLETE
+                      Group By Operator
+                        aggregations: count()
+                        keys: true (type: boolean)
+                        minReductionHashAggr: 0.5
+                        mode: hash
+                        outputColumnNames: _col0, _col1
+                        Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          key expressions: _col0 (type: boolean)
+                          null sort order: z
+                          sort order: +
+                          Map-reduce partition columns: _col0 (type: boolean)
+                          Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                          value expressions: _col1 (type: bigint)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 5 
+        Map 6 
             Map Operator Tree:
                 TableScan
                   alias: src
@@ -2816,33 +2736,6 @@ STAGE PLANS:
                           Statistics: Num rows: 250 Data size: 22750 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 6 
-            Map Operator Tree:
-                TableScan
-                  alias: s1
-                  filterExpr: (key = '90') (type: boolean)
-                  Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: (key = '90') (type: boolean)
-                    Statistics: Num rows: 2 Data size: 174 Basic stats: COMPLETE Column stats: COMPLETE
-                    Select Operator
-                      Statistics: Num rows: 2 Data size: 174 Basic stats: COMPLETE Column stats: COMPLETE
-                      Group By Operator
-                        aggregations: count()
-                        keys: true (type: boolean)
-                        minReductionHashAggr: 0.5
-                        mode: hash
-                        outputColumnNames: _col0, _col1
-                        Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
-                        Reduce Output Operator
-                          key expressions: _col0 (type: boolean)
-                          null sort order: z
-                          sort order: +
-                          Map-reduce partition columns: _col0 (type: boolean)
-                          Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
-                          value expressions: _col1 (type: bigint)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
@@ -2905,7 +2798,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 7 
+        Reducer 5 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -2994,17 +2887,17 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 6 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
-        Reducer 4 <- Reducer 3 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE)
-        Reducer 7 <- Map 6 (SIMPLE_EDGE)
+        Reducer 4 <- Reducer 3 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
+        Reducer 5 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: src
-                  filterExpr: (key is not null and value is not null) (type: boolean)
+                  filterExpr: ((key is not null and value is not null) or (key = '90')) (type: boolean)
                   Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key is not null and value is not null) (type: boolean)
@@ -3019,9 +2912,28 @@ STAGE PLANS:
                         sort order: ++
                         Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
                         Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: (key = '90') (type: boolean)
+                    Statistics: Num rows: 2 Data size: 174 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      Statistics: Num rows: 2 Data size: 174 Basic stats: COMPLETE Column stats: COMPLETE
+                      Group By Operator
+                        aggregations: count()
+                        keys: true (type: boolean)
+                        minReductionHashAggr: 0.5
+                        mode: hash
+                        outputColumnNames: _col0, _col1
+                        Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          key expressions: _col0 (type: boolean)
+                          null sort order: z
+                          sort order: +
+                          Map-reduce partition columns: _col0 (type: boolean)
+                          Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                          value expressions: _col1 (type: bigint)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 5 
+        Map 6 
             Map Operator Tree:
                 TableScan
                   alias: sc
@@ -3048,33 +2960,6 @@ STAGE PLANS:
                           Statistics: Num rows: 250 Data size: 44500 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 6 
-            Map Operator Tree:
-                TableScan
-                  alias: s1
-                  filterExpr: (key = '90') (type: boolean)
-                  Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: (key = '90') (type: boolean)
-                    Statistics: Num rows: 2 Data size: 174 Basic stats: COMPLETE Column stats: COMPLETE
-                    Select Operator
-                      Statistics: Num rows: 2 Data size: 174 Basic stats: COMPLETE Column stats: COMPLETE
-                      Group By Operator
-                        aggregations: count()
-                        keys: true (type: boolean)
-                        minReductionHashAggr: 0.5
-                        mode: hash
-                        outputColumnNames: _col0, _col1
-                        Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
-                        Reduce Output Operator
-                          key expressions: _col0 (type: boolean)
-                          null sort order: z
-                          sort order: +
-                          Map-reduce partition columns: _col0 (type: boolean)
-                          Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
-                          value expressions: _col1 (type: bigint)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
@@ -3137,7 +3022,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 7 
+        Reducer 5 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -3226,16 +3111,15 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
-        Reducer 5 <- Map 4 (SIMPLE_EDGE)
+        Reducer 4 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: part
-                  filterExpr: p_size is not null (type: boolean)
                   Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: p_size is not null (type: boolean)
@@ -3251,13 +3135,6 @@ STAGE PLANS:
                         Map-reduce partition columns: _col9 (type: int)
                         Statistics: Num rows: 26 Data size: 16198 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 4 
-            Map Operator Tree:
-                TableScan
-                  alias: part
-                  Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: p_type (type: string), p_size (type: int)
                     outputColumnNames: p_type, p_size
@@ -3309,7 +3186,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 5 
+        Reducer 4 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -3383,16 +3260,15 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
-        Reducer 5 <- Map 4 (SIMPLE_EDGE)
+        Reducer 4 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: part
-                  filterExpr: p_size is not null (type: boolean)
                   Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: p_size is not null (type: boolean)
@@ -3408,13 +3284,6 @@ STAGE PLANS:
                         Map-reduce partition columns: _col9 (type: int)
                         Statistics: Num rows: 26 Data size: 16198 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 4 
-            Map Operator Tree:
-                TableScan
-                  alias: part
-                  Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: p_type (type: string), p_size (type: int)
                     outputColumnNames: p_type, p_size
@@ -3475,7 +3344,7 @@ STAGE PLANS:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                         serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 5 
+        Reducer 4 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -3812,15 +3681,15 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-        Reducer 4 <- Map 3 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: part
-                  filterExpr: (p_size is not null and p_name is not null) (type: boolean)
+                  filterExpr: ((p_size is not null and p_name is not null) or (p_type is not null and p_name is not null)) (type: boolean)
                   Statistics: Num rows: 26 Data size: 3354 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (p_size is not null and p_name is not null) (type: boolean)
@@ -3836,14 +3705,6 @@ STAGE PLANS:
                         Map-reduce partition columns: _col1 (type: string), _col2 (type: int)
                         Statistics: Num rows: 26 Data size: 3354 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 3 
-            Map Operator Tree:
-                TableScan
-                  alias: p
-                  filterExpr: (p_type is not null and p_name is not null) (type: boolean)
-                  Statistics: Num rows: 26 Data size: 5850 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (p_type is not null and p_name is not null) (type: boolean)
                     Statistics: Num rows: 26 Data size: 5850 Basic stats: COMPLETE Column stats: COMPLETE
@@ -3860,7 +3721,7 @@ STAGE PLANS:
                         value expressions: _col0 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 5 
+        Map 4 
             Map Operator Tree:
                 TableScan
                   alias: pp
@@ -3900,7 +3761,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 4 
+        Reducer 3 
             Execution mode: llap
             Reduce Operator Tree:
               Merge Join Operator
@@ -3987,15 +3848,15 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-        Reducer 4 <- Map 3 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: part
-                  filterExpr: (p_size is not null and p_name is not null) (type: boolean)
+                  filterExpr: ((p_size is not null and p_name is not null) or (p_size is not null and p_type is not null and p_name is not null)) (type: boolean)
                   Statistics: Num rows: 26 Data size: 3354 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (p_size is not null and p_name is not null) (type: boolean)
@@ -4011,14 +3872,6 @@ STAGE PLANS:
                         Map-reduce partition columns: _col1 (type: string), _col2 (type: int)
                         Statistics: Num rows: 26 Data size: 3354 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 3 
-            Map Operator Tree:
-                TableScan
-                  alias: p
-                  filterExpr: (p_size is not null and p_type is not null and p_name is not null) (type: boolean)
-                  Statistics: Num rows: 26 Data size: 5954 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (p_size is not null and p_type is not null and p_name is not null) (type: boolean)
                     Statistics: Num rows: 26 Data size: 5954 Basic stats: COMPLETE Column stats: COMPLETE
@@ -4035,7 +3888,7 @@ STAGE PLANS:
                         value expressions: _col0 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 5 
+        Map 4 
             Map Operator Tree:
                 TableScan
                   alias: pp
@@ -4075,7 +3928,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 4 
+        Reducer 3 
             Execution mode: llap
             Reduce Operator Tree:
               Merge Join Operator
@@ -4164,15 +4017,15 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-        Reducer 4 <- Map 3 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: part
-                  filterExpr: (p_type is not null and p_size is not null and p_name is not null) (type: boolean)
+                  filterExpr: ((p_type is not null and p_size is not null and p_name is not null) or (p_type is not null and p_name is not null)) (type: boolean)
                   Statistics: Num rows: 26 Data size: 6058 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (p_type is not null and p_size is not null and p_name is not null) (type: boolean)
@@ -4188,14 +4041,6 @@ STAGE PLANS:
                         Map-reduce partition columns: _col1 (type: string), _col2 (type: string), _col3 (type: int)
                         Statistics: Num rows: 26 Data size: 6058 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 3 
-            Map Operator Tree:
-                TableScan
-                  alias: p
-                  filterExpr: (p_type is not null and p_name is not null) (type: boolean)
-                  Statistics: Num rows: 26 Data size: 5850 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (p_type is not null and p_name is not null) (type: boolean)
                     Statistics: Num rows: 26 Data size: 5850 Basic stats: COMPLETE Column stats: COMPLETE
@@ -4212,7 +4057,7 @@ STAGE PLANS:
                         value expressions: _col0 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 5 
+        Map 4 
             Map Operator Tree:
                 TableScan
                   alias: pp
@@ -4252,7 +4097,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 4 
+        Reducer 3 
             Execution mode: llap
             Reduce Operator Tree:
               Merge Join Operator
@@ -4305,15 +4150,15 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-        Reducer 4 <- Map 3 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: part
-                  filterExpr: (p_size is not null and p_name is not null) (type: boolean)
+                  filterExpr: ((p_size is not null and p_name is not null) or (p_type is not null and p_name is not null)) (type: boolean)
                   Statistics: Num rows: 26 Data size: 3354 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (p_size is not null and p_name is not null) (type: boolean)
@@ -4329,14 +4174,6 @@ STAGE PLANS:
                         Map-reduce partition columns: _col1 (type: string), _col2 (type: int)
                         Statistics: Num rows: 26 Data size: 3354 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 3 
-            Map Operator Tree:
-                TableScan
-                  alias: p
-                  filterExpr: (p_type is not null and p_name is not null) (type: boolean)
-                  Statistics: Num rows: 26 Data size: 5850 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (p_type is not null and p_name is not null) (type: boolean)
                     Statistics: Num rows: 26 Data size: 5850 Basic stats: COMPLETE Column stats: COMPLETE
@@ -4353,7 +4190,7 @@ STAGE PLANS:
                         value expressions: _col0 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 5 
+        Map 4 
             Map Operator Tree:
                 TableScan
                   alias: pp
@@ -4393,7 +4230,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 4 
+        Reducer 3 
             Execution mode: llap
             Reduce Operator Tree:
               Merge Join Operator
@@ -4480,17 +4317,17 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
-        Reducer 5 <- Map 4 (SIMPLE_EDGE)
-        Reducer 6 <- Map 4 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
+        Reducer 4 <- Map 1 (SIMPLE_EDGE)
+        Reducer 5 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: part
-                  filterExpr: (p_type is not null and UDFToLong(p_size) is not null) (type: boolean)
+                  filterExpr: ((p_type is not null and UDFToLong(p_size) is not null) or p_type is not null) (type: boolean)
                   Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (p_type is not null and UDFToLong(p_size) is not null) (type: boolean)
@@ -4506,14 +4343,6 @@ STAGE PLANS:
                         Map-reduce partition columns: _col4 (type: string)
                         Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 4 
-            Map Operator Tree:
-                TableScan
-                  alias: pp
-                  filterExpr: p_type is not null (type: boolean)
-                  Statistics: Num rows: 26 Data size: 2704 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: p_type is not null (type: boolean)
                     Statistics: Num rows: 26 Data size: 2704 Basic stats: COMPLETE Column stats: COMPLETE
@@ -4588,7 +4417,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 5 
+        Reducer 4 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -4609,7 +4438,7 @@ STAGE PLANS:
                     Map-reduce partition columns: _col0 (type: string)
                     Statistics: Num rows: 6 Data size: 672 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col1 (type: bigint)
-        Reducer 6 
+        Reducer 5 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -4668,15 +4497,15 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-        Reducer 4 <- Map 3 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: part
-                  filterExpr: (p_partkey is not null and UDFToDouble(p_size) is not null) (type: boolean)
+                  filterExpr: ((p_partkey is not null and UDFToDouble(p_size) is not null) or p_partkey is not null) (type: boolean)
                   Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (p_partkey is not null and UDFToDouble(p_size) is not null) (type: boolean)
@@ -4692,14 +4521,6 @@ STAGE PLANS:
                         Map-reduce partition columns: _col0 (type: int), UDFToDouble(_col5) (type: double)
                         Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 3 
-            Map Operator Tree:
-                TableScan
-                  alias: pp
-                  filterExpr: p_partkey is not null (type: boolean)
-                  Statistics: Num rows: 26 Data size: 208 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: p_partkey is not null (type: boolean)
                     Statistics: Num rows: 26 Data size: 208 Basic stats: COMPLETE Column stats: COMPLETE
@@ -4737,7 +4558,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 4 
+        Reducer 3 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -4806,7 +4627,7 @@ POSTHOOK: Input: default@part
 85768	almond antique chartreuse lavender yellow	Manufacturer#1	Brand#12	LARGE BRUSHED STEEL	34	SM BAG	1753.76	refull
 86428	almond aquamarine burnished black steel	Manufacturer#1	Brand#12	STANDARD ANODIZED STEEL	28	WRAP BAG	1414.42	arefully 
 90681	almond antique chartreuse khaki white	Manufacturer#3	Brand#31	MEDIUM BURNISHED TIN	17	SM CASE	1671.68	are slyly after the sl
-Warning: Shuffle Join MERGEJOIN[53][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product
+Warning: Shuffle Join MERGEJOIN[53][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: explain select * from part where p_size in (select min(pp.p_size) from part pp where pp.p_partkey > part.p_partkey)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
@@ -4824,17 +4645,17 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
-        Reducer 4 <- Map 3 (XPROD_EDGE), Reducer 7 (XPROD_EDGE)
-        Reducer 5 <- Reducer 4 (SIMPLE_EDGE)
-        Reducer 7 <- Map 6 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (XPROD_EDGE), Reducer 5 (XPROD_EDGE)
+        Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
+        Reducer 5 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: part
-                  filterExpr: (p_partkey is not null and p_size is not null) (type: boolean)
+                  filterExpr: ((p_partkey is not null and p_size is not null) or p_partkey is not null or p_partkey is not null) (type: boolean)
                   Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (p_partkey is not null and p_size is not null) (type: boolean)
@@ -4850,14 +4671,6 @@ STAGE PLANS:
                         Map-reduce partition columns: _col0 (type: int), _col5 (type: int)
                         Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col6 (type: string), _col7 (type: double), _col8 (type: string)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 3 
-            Map Operator Tree:
-                TableScan
-                  alias: pp
-                  filterExpr: p_partkey is not null (type: boolean)
-                  Statistics: Num rows: 26 Data size: 208 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: p_partkey is not null (type: boolean)
                     Statistics: Num rows: 26 Data size: 208 Basic stats: COMPLETE Column stats: COMPLETE
@@ -4870,14 +4683,6 @@ STAGE PLANS:
                         sort order: 
                         Statistics: Num rows: 26 Data size: 208 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int), _col1 (type: int)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 6 
-            Map Operator Tree:
-                TableScan
-                  alias: part
-                  filterExpr: p_partkey is not null (type: boolean)
-                  Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: p_partkey is not null (type: boolean)
                     Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
@@ -4913,7 +4718,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 4 
+        Reducer 3 
             Execution mode: llap
             Reduce Operator Tree:
               Merge Join Operator
@@ -4943,7 +4748,7 @@ STAGE PLANS:
                       Map-reduce partition columns: _col0 (type: int)
                       Statistics: Num rows: 13 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
                       value expressions: _col1 (type: int)
-        Reducer 5 
+        Reducer 4 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -4967,7 +4772,7 @@ STAGE PLANS:
                       sort order: ++
                       Map-reduce partition columns: _col0 (type: int), _col1 (type: int)
                       Statistics: Num rows: 6 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
-        Reducer 7 
+        Reducer 5 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -4987,7 +4792,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[53][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product
+Warning: Shuffle Join MERGEJOIN[53][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: select * from part where p_size in (select min(pp.p_size) from part pp where pp.p_partkey > part.p_partkey)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
@@ -5016,12 +4821,12 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE)
-        Reducer 4 <- Reducer 3 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE)
-        Reducer 6 <- Map 5 (SIMPLE_EDGE)
-        Reducer 7 <- Map 5 (SIMPLE_EDGE)
-        Reducer 8 <- Map 5 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
+        Reducer 4 <- Reducer 3 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE)
+        Reducer 5 <- Map 1 (SIMPLE_EDGE)
+        Reducer 6 <- Map 1 (SIMPLE_EDGE)
+        Reducer 7 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -5040,14 +4845,6 @@ STAGE PLANS:
                       Map-reduce partition columns: _col4 (type: string)
                       Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
                       value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 5 
-            Map Operator Tree:
-                TableScan
-                  alias: pp
-                  filterExpr: p_type is not null (type: boolean)
-                  Statistics: Num rows: 26 Data size: 2704 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: p_type is not null (type: boolean)
                     Statistics: Num rows: 26 Data size: 2704 Basic stats: COMPLETE Column stats: COMPLETE
@@ -5161,7 +4958,7 @@ STAGE PLANS:
                           input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                           output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                           serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 6 
+        Reducer 5 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -5182,7 +4979,7 @@ STAGE PLANS:
                     Map-reduce partition columns: _col0 (type: string)
                     Statistics: Num rows: 6 Data size: 672 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col1 (type: bigint)
-        Reducer 7 
+        Reducer 6 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -5204,7 +5001,7 @@ STAGE PLANS:
                     Map-reduce partition columns: _col0 (type: string)
                     Statistics: Num rows: 6 Data size: 720 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col1 (type: bigint), _col2 (type: bigint)
-        Reducer 8 
+        Reducer 7 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -5282,11 +5079,11 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE)
-        Reducer 4 <- Reducer 3 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE)
-        Reducer 6 <- Map 5 (SIMPLE_EDGE)
-        Reducer 8 <- Map 7 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
+        Reducer 4 <- Reducer 3 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
+        Reducer 5 <- Map 1 (SIMPLE_EDGE)
+        Reducer 6 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -5305,14 +5102,6 @@ STAGE PLANS:
                       Map-reduce partition columns: _col0 (type: int)
                       Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
                       value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 5 
-            Map Operator Tree:
-                TableScan
-                  alias: pp
-                  filterExpr: p_partkey is not null (type: boolean)
-                  Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: p_partkey is not null (type: boolean)
                     Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
@@ -5328,14 +5117,6 @@ STAGE PLANS:
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 13 Data size: 52 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 7 
-            Map Operator Tree:
-                TableScan
-                  alias: pp
-                  filterExpr: p_partkey is not null (type: boolean)
-                  Statistics: Num rows: 26 Data size: 208 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: p_partkey is not null (type: boolean)
                     Statistics: Num rows: 26 Data size: 208 Basic stats: COMPLETE Column stats: COMPLETE
@@ -5423,7 +5204,7 @@ STAGE PLANS:
                           input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                           output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                           serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 6 
+        Reducer 5 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -5444,7 +5225,7 @@ STAGE PLANS:
                     Map-reduce partition columns: _col0 (type: int)
                     Statistics: Num rows: 6 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col1 (type: bigint)
-        Reducer 8 
+        Reducer 6 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -5907,7 +5688,7 @@ POSTHOOK: query: drop table tt_n2
 POSTHOOK: type: DROPTABLE
 POSTHOOK: Input: default@tt_n2
 POSTHOOK: Output: default@tt_n2
-Warning: Shuffle Join MERGEJOIN[49][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product
+Warning: Shuffle Join MERGEJOIN[49][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: explain select * from part where p_size IN (select max(p_size) from part p where p.p_type <> part.p_name)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
@@ -5925,17 +5706,16 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
-        Reducer 4 <- Map 3 (XPROD_EDGE), Reducer 7 (XPROD_EDGE)
-        Reducer 5 <- Reducer 4 (SIMPLE_EDGE)
-        Reducer 7 <- Map 6 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (XPROD_EDGE), Reducer 5 (XPROD_EDGE)
+        Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
+        Reducer 5 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: part
-                  filterExpr: (p_name is not null and p_size is not null) (type: boolean)
                   Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (p_name is not null and p_size is not null) (type: boolean)
@@ -5951,13 +5731,6 @@ STAGE PLANS:
                         Map-reduce partition columns: _col1 (type: string), _col5 (type: int)
                         Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col6 (type: string), _col7 (type: double), _col8 (type: string)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 3 
-            Map Operator Tree:
-                TableScan
-                  alias: p
-                  Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: p_type (type: string), p_size (type: int)
                     outputColumnNames: _col0, _col1
@@ -5967,14 +5740,6 @@ STAGE PLANS:
                       sort order: 
                       Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE
                       value expressions: _col0 (type: string), _col1 (type: int)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 6 
-            Map Operator Tree:
-                TableScan
-                  alias: part
-                  filterExpr: p_name is not null (type: boolean)
-                  Statistics: Num rows: 26 Data size: 3146 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: p_name is not null (type: boolean)
                     Statistics: Num rows: 26 Data size: 3146 Basic stats: COMPLETE Column stats: COMPLETE
@@ -6010,7 +5775,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 4 
+        Reducer 3 
             Execution mode: llap
             Reduce Operator Tree:
               Merge Join Operator
@@ -6040,7 +5805,7 @@ STAGE PLANS:
                       Map-reduce partition columns: _col0 (type: string)
                       Statistics: Num rows: 13 Data size: 1625 Basic stats: COMPLETE Column stats: COMPLETE
                       value expressions: _col1 (type: int)
-        Reducer 5 
+        Reducer 4 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -6064,7 +5829,7 @@ STAGE PLANS:
                       sort order: ++
                       Map-reduce partition columns: _col0 (type: string), _col1 (type: int)
                       Statistics: Num rows: 6 Data size: 750 Basic stats: COMPLETE Column stats: COMPLETE
-        Reducer 7 
+        Reducer 5 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -6084,7 +5849,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[49][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product
+Warning: Shuffle Join MERGEJOIN[49][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: select * from part where p_size IN (select max(p_size) from part p where p.p_type <> part.p_name)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
@@ -6094,7 +5859,7 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@part
 #### A masked pattern was here ####
 15103	almond aquamarine dodger light gainsboro	Manufacturer#5	Brand#53	ECONOMY BURNISHED STEEL	46	LG PACK	1018.1	packages hinder carefu
-Warning: Shuffle Join MERGEJOIN[71][tables = [$hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 5' is a cross product
+Warning: Shuffle Join MERGEJOIN[71][tables = [$hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product
 PREHOOK: query: explain select * from part where p_size IN (select pp.p_size from part p join part pp on pp.p_type = p.p_type where part.p_type <> p.p_name)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
@@ -6112,17 +5877,17 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
-        Reducer 4 <- Map 3 (SIMPLE_EDGE), Map 6 (SIMPLE_EDGE)
-        Reducer 5 <- Reducer 4 (XPROD_EDGE), Reducer 8 (XPROD_EDGE)
-        Reducer 8 <- Map 7 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE), Map 6 (SIMPLE_EDGE)
+        Reducer 4 <- Reducer 3 (XPROD_EDGE), Reducer 5 (XPROD_EDGE)
+        Reducer 5 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: part
-                  filterExpr: (p_type is not null and p_size is not null) (type: boolean)
+                  filterExpr: ((p_type is not null and p_size is not null) or p_type is not null or p_type is not null) (type: boolean)
                   Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (p_type is not null and p_size is not null) (type: boolean)
@@ -6138,14 +5903,6 @@ STAGE PLANS:
                         Map-reduce partition columns: _col4 (type: string), _col5 (type: int)
                         Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col6 (type: string), _col7 (type: double), _col8 (type: string)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 3 
-            Map Operator Tree:
-                TableScan
-                  alias: p
-                  filterExpr: p_type is not null (type: boolean)
-                  Statistics: Num rows: 26 Data size: 5850 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: p_type is not null (type: boolean)
                     Statistics: Num rows: 26 Data size: 5850 Basic stats: COMPLETE Column stats: COMPLETE
@@ -6160,6 +5917,21 @@ STAGE PLANS:
                         Map-reduce partition columns: _col1 (type: string)
                         Statistics: Num rows: 26 Data size: 5850 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: string)
+                  Filter Operator
+                    predicate: p_type is not null (type: boolean)
+                    Statistics: Num rows: 26 Data size: 2704 Basic stats: COMPLETE Column stats: COMPLETE
+                    Group By Operator
+                      keys: p_type (type: string)
+                      minReductionHashAggr: 0.5
+                      mode: hash
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 13 Data size: 1352 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 13 Data size: 1352 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 6 
@@ -6184,29 +5956,6 @@ STAGE PLANS:
                         value expressions: _col1 (type: int)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 7 
-            Map Operator Tree:
-                TableScan
-                  alias: part
-                  filterExpr: p_type is not null (type: boolean)
-                  Statistics: Num rows: 26 Data size: 2704 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: p_type is not null (type: boolean)
-                    Statistics: Num rows: 26 Data size: 2704 Basic stats: COMPLETE Column stats: COMPLETE
-                    Group By Operator
-                      keys: p_type (type: string)
-                      minReductionHashAggr: 0.5
-                      mode: hash
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 13 Data size: 1352 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: string)
-                        null sort order: z
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: string)
-                        Statistics: Num rows: 13 Data size: 1352 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
@@ -6225,7 +5974,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 4 
+        Reducer 3 
             Execution mode: llap
             Reduce Operator Tree:
               Merge Join Operator
@@ -6241,7 +5990,7 @@ STAGE PLANS:
                   sort order: 
                   Statistics: Num rows: 28 Data size: 3500 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col0 (type: string), _col3 (type: int)
-        Reducer 5 
+        Reducer 4 
             Execution mode: llap
             Reduce Operator Tree:
               Merge Join Operator
@@ -6269,7 +6018,7 @@ STAGE PLANS:
                       sort order: ++
                       Map-reduce partition columns: _col0 (type: string), _col1 (type: int)
                       Statistics: Num rows: 182 Data size: 19656 Basic stats: COMPLETE Column stats: COMPLETE
-        Reducer 8 
+        Reducer 5 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -6289,7 +6038,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[71][tables = [$hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 5' is a cross product
+Warning: Shuffle Join MERGEJOIN[71][tables = [$hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product
 PREHOOK: query: select * from part where p_size IN (select pp.p_size from part p join part pp on pp.p_type = p.p_type where part.p_type <> p.p_name)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
diff --git a/ql/src/test/results/clientpositive/llap/subquery_in_having.q.out b/ql/src/test/results/clientpositive/llap/subquery_in_having.q.out
index d4b2967..bf8c197 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_in_having.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_in_having.q.out
@@ -575,7 +575,7 @@ STAGE PLANS:
 #### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
-        Reducer 3 <- Map 7 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
         Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
... 20606 lines suppressed ...