You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by vg...@apache.org on 2019/04/08 17:13:40 UTC

[hive] branch master updated: HIVE-21231: HiveJoinAddNotNullRule support for range predicates (Miklos Gergely, reviewed by Vineet Garg)

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

vgarg 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 1ddecfc  HIVE-21231: HiveJoinAddNotNullRule support for range predicates (Miklos Gergely, reviewed by Vineet Garg)
1ddecfc is described below

commit 1ddecfcd8d47eabce83708157a59f968f971fe8a
Author: Miklos Gergely <mg...@hortonworks.com>
AuthorDate: Mon Apr 8 10:11:33 2019 -0700

    HIVE-21231: HiveJoinAddNotNullRule support for range predicates (Miklos Gergely, reviewed by Vineet Garg)
---
 .../hive/ql/optimizer/calcite/HiveCalciteUtil.java |    6 +-
 .../calcite/rules/HiveJoinAddNotNullRule.java      |  130 +-
 .../optimizer/calcite/rules/HiveRulesRegistry.java |    2 +-
 .../clientpositive/join_by_range_rule_not_null.q   |    9 +
 .../subquery_scalar_multi_rows.q.out               |    4 +-
 .../test/results/clientpositive/interval_3.q.out   |   34 +-
 ql/src/test/results/clientpositive/join43.q.out    |   16 +-
 .../join_by_range_rule_not_null.q.out              |  430 ++++++
 .../test/results/clientpositive/join_merging.q.out |    4 +-
 .../results/clientpositive/llap/cross_prod_1.q.out |   36 +-
 .../llap/dynamic_semijoin_reduction.q.out          |    8 +-
 .../llap/groupby_groupingset_bug.q.out             |  102 +-
 .../results/clientpositive/llap/semijoin.q.out     |    9 +-
 .../clientpositive/llap/subquery_corr.q.out        |   12 +-
 .../results/clientpositive/llap/subquery_in.q.out  |   27 +-
 .../clientpositive/llap/subquery_notin.q.out       |    2 +-
 .../clientpositive/llap/subquery_scalar.q.out      | 1107 ++++++++++-----
 .../clientpositive/llap/subquery_select.q.out      |   39 +-
 .../clientpositive/llap/transitive_not_null.q.out  |    4 +-
 .../results/clientpositive/perf/spark/query1.q.out |  140 +-
 .../clientpositive/perf/spark/query23.q.out        |  140 +-
 .../clientpositive/perf/spark/query24.q.out        |   66 +-
 .../clientpositive/perf/spark/query30.q.out        |   32 +-
 .../clientpositive/perf/spark/query32.q.out        |   98 +-
 .../clientpositive/perf/spark/query44.q.out        |  369 +++--
 .../clientpositive/perf/spark/query54.q.out        |   58 +-
 .../results/clientpositive/perf/spark/query6.q.out |   43 +-
 .../clientpositive/perf/spark/query65.q.out        |   32 +-
 .../clientpositive/perf/spark/query72.q.out        |   16 +-
 .../clientpositive/perf/spark/query81.q.out        |   32 +-
 .../clientpositive/perf/spark/query92.q.out        |   98 +-
 .../clientpositive/perf/tez/cbo_ext_query1.q.out   |   92 +-
 .../clientpositive/perf/tez/cbo_query1.q.out       |   46 +-
 .../clientpositive/perf/tez/cbo_query14.q.out      |  543 ++++----
 .../clientpositive/perf/tez/cbo_query23.q.out      |  104 +-
 .../clientpositive/perf/tez/cbo_query24.q.out      |  114 +-
 .../clientpositive/perf/tez/cbo_query30.q.out      |   37 +-
 .../clientpositive/perf/tez/cbo_query32.q.out      |   44 +-
 .../clientpositive/perf/tez/cbo_query54.q.out      |    6 +-
 .../clientpositive/perf/tez/cbo_query6.q.out       |   11 +-
 .../clientpositive/perf/tez/cbo_query64.q.out      |  316 ++---
 .../clientpositive/perf/tez/cbo_query65.q.out      |   41 +-
 .../clientpositive/perf/tez/cbo_query72.q.out      |    8 +-
 .../clientpositive/perf/tez/cbo_query81.q.out      |   37 +-
 .../clientpositive/perf/tez/cbo_query92.q.out      |   44 +-
 .../perf/tez/constraints/cbo_ext_query1.q.out      |   70 +-
 .../perf/tez/constraints/cbo_query1.q.out          |   34 +-
 .../perf/tez/constraints/cbo_query14.q.out         |  549 ++++----
 .../perf/tez/constraints/cbo_query23.q.out         |   66 +-
 .../perf/tez/constraints/cbo_query24.q.out         |  110 +-
 .../perf/tez/constraints/cbo_query30.q.out         |   37 +-
 .../perf/tez/constraints/cbo_query32.q.out         |   44 +-
 .../perf/tez/constraints/cbo_query54.q.out         |   15 +-
 .../perf/tez/constraints/cbo_query6.q.out          |   43 +-
 .../perf/tez/constraints/cbo_query64.q.out         |  242 ++--
 .../perf/tez/constraints/cbo_query65.q.out         |   37 +-
 .../perf/tez/constraints/cbo_query72.q.out         |   10 +-
 .../perf/tez/constraints/cbo_query81.q.out         |   37 +-
 .../perf/tez/constraints/cbo_query92.q.out         |   44 +-
 .../perf/tez/constraints/mv_query44.q.out          |  220 +--
 .../perf/tez/constraints/query1.q.out              |  196 +--
 .../perf/tez/constraints/query14.q.out             | 1436 +++++++++----------
 .../perf/tez/constraints/query23.q.out             |  416 +++---
 .../perf/tez/constraints/query24.q.out             |  346 ++---
 .../perf/tez/constraints/query30.q.out             |  212 +--
 .../perf/tez/constraints/query32.q.out             |  188 ++-
 .../perf/tez/constraints/query44.q.out             |  220 +--
 .../perf/tez/constraints/query54.q.out             |  305 ++--
 .../perf/tez/constraints/query6.q.out              |  287 ++--
 .../perf/tez/constraints/query64.q.out             |  838 +++++------
 .../perf/tez/constraints/query65.q.out             |  146 +-
 .../perf/tez/constraints/query72.q.out             |  170 +--
 .../perf/tez/constraints/query81.q.out             |  208 +--
 .../perf/tez/constraints/query92.q.out             |  188 ++-
 .../results/clientpositive/perf/tez/query1.q.out   |  198 +--
 .../results/clientpositive/perf/tez/query14.q.out  | 1450 ++++++++++----------
 .../results/clientpositive/perf/tez/query23.q.out  |  448 +++---
 .../results/clientpositive/perf/tez/query24.q.out  |  354 ++---
 .../results/clientpositive/perf/tez/query30.q.out  |  212 +--
 .../results/clientpositive/perf/tez/query32.q.out  |  188 ++-
 .../results/clientpositive/perf/tez/query44.q.out  |  208 +--
 .../results/clientpositive/perf/tez/query54.q.out  |  130 +-
 .../results/clientpositive/perf/tez/query6.q.out   |  164 +--
 .../results/clientpositive/perf/tez/query64.q.out  |  880 ++++++------
 .../results/clientpositive/perf/tez/query65.q.out  |  180 +--
 .../results/clientpositive/perf/tez/query72.q.out  |   48 +-
 .../results/clientpositive/perf/tez/query81.q.out  |  208 +--
 .../results/clientpositive/perf/tez/query92.q.out  |  188 ++-
 .../clientpositive/spark/join_merging.q.out        |    4 +-
 .../results/clientpositive/spark/semijoin.q.out    |    4 +-
 .../results/clientpositive/spark/subquery_in.q.out |   24 +-
 .../clientpositive/spark/subquery_notin.q.out      |    4 +-
 .../clientpositive/spark/subquery_scalar.q.out     | 1098 +++++++++------
 .../clientpositive/spark/subquery_select.q.out     |   35 +-
 94 files changed, 9412 insertions(+), 7875 deletions(-)

diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveCalciteUtil.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveCalciteUtil.java
index b480fe6..9fed1fd 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveCalciteUtil.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveCalciteUtil.java
@@ -491,6 +491,10 @@ public class HiveCalciteUtil {
       this.projsJoinKeysInJoinSchema = projsJoinKeysInJoinSchemaBuilder.build();
     }
 
+    public SqlKind getComparisonType() {
+      return comparisonType;
+    }
+
     public List<RexNode> getJoinExprs(int input) {
       return this.joinKeyExprs.get(input);
     }
@@ -537,7 +541,7 @@ public class HiveCalciteUtil {
 
       // 1. Split leaf join predicate to expressions from left, right
       RexNode otherConditions = HiveRelOptUtil.splitHiveJoinCondition(systemFieldList, inputs, pe,
-          joinExprs, filterNulls, null);
+          joinExprs, filterNulls, new ArrayList<SqlOperator>());
 
       if (otherConditions.isAlwaysTrue()) {
         // 2. Collect child projection indexes used
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinAddNotNullRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinAddNotNullRule.java
index 48e2cc6..4e66de3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinAddNotNullRule.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinAddNotNullRule.java
@@ -21,16 +21,17 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Set;
 
-import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinRelType;
 import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.core.RelFactories.FilterFactory;
 import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.hadoop.hive.ql.optimizer.calcite.CalciteSemanticException;
 import org.apache.hadoop.hive.ql.optimizer.calcite.HiveCalciteUtil;
@@ -43,6 +44,10 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSemiJoin;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 
+/**
+ * Responsible for adding not null rules to joins, when the declaration of a join implies that some coulmns
+ * may not be null.
+ */
 public final class HiveJoinAddNotNullRule extends RelOptRule {
 
   public static final HiveJoinAddNotNullRule INSTANCE_JOIN =
@@ -68,34 +73,8 @@ public final class HiveJoinAddNotNullRule extends RelOptRule {
 
   @Override
   public void onMatch(RelOptRuleCall call) {
-    final Join join = call.rel(0);
-    RelNode lChild = join.getLeft();
-    RelNode rChild = join.getRight();
-
-    HiveRulesRegistry registry = call.getPlanner().getContext().unwrap(HiveRulesRegistry.class);
-    assert registry != null;
-
-
-    if (join.getCondition().isAlwaysTrue()) {
-      return;
-    }
-
-    boolean genPredOnLeftSide = false, genPredOnRightSide = false;
-
-    switch (join.getJoinType()) {
-    case INNER:
-      genPredOnLeftSide = true;
-      genPredOnRightSide = true;
-      break;
-    case LEFT:
-      genPredOnLeftSide = false;
-      genPredOnRightSide = true;
-      break;
-    case RIGHT:
-      genPredOnLeftSide = true;
-      genPredOnRightSide = false;
-      break;
-    default:
+    Join join = call.rel(0);
+    if (join.getJoinType() == JoinRelType.FULL || join.getCondition().isAlwaysTrue()) {
       return;
     }
 
@@ -106,57 +85,27 @@ public final class HiveJoinAddNotNullRule extends RelOptRule {
       return;
     }
 
-    List<RexNode> leftJoinExprsList = new ArrayList<>();
-    List<RexNode> rightJoinExprsList = new ArrayList<>();
-    for (JoinLeafPredicateInfo joinLeafPredicateInfo : joinPredInfo.getEquiJoinPredicateElements()) {
-      leftJoinExprsList.addAll(joinLeafPredicateInfo.getJoinExprs(0));
-      rightJoinExprsList.addAll(joinLeafPredicateInfo.getJoinExprs(1));
-    }
-
-    // Build not null conditions
-    final RelOptCluster cluster = join.getCluster();
-    final RexBuilder rexBuilder = join.getCluster().getRexBuilder();
-
-
-    RexNode newLeftPredicate = rexBuilder.makeLiteral(true);
-    RexNode newRightPredicate = rexBuilder.makeLiteral(true);
+    HiveRulesRegistry registry = call.getPlanner().getContext().unwrap(HiveRulesRegistry.class);
+    assert registry != null;
 
-    // generate predicate on left side
     Set<String> leftPushedPredicates = Sets.newHashSet(registry.getPushedPredicates(join, 0));
-    if (genPredOnLeftSide) {
-      List<RexNode> newLeftConditions = getNotNullConditions(cluster,
-                                                             rexBuilder, leftJoinExprsList, leftPushedPredicates);
-      newLeftPredicate = RexUtil.composeConjunction(rexBuilder, newLeftConditions, false);
-    }
-
-    // generate predicate on right side
     Set<String> rightPushedPredicates = Sets.newHashSet(registry.getPushedPredicates(join, 1));
-    if (genPredOnRightSide) {
-      List<RexNode> newRightConditions = getNotNullConditions(cluster,
-                                                              rexBuilder, rightJoinExprsList, rightPushedPredicates);
-      newRightPredicate = RexUtil.composeConjunction(rexBuilder, newRightConditions, false);
-    }
 
-    boolean isNewLeftPredicateAlwaysTrue = newLeftPredicate.isAlwaysTrue();
-    boolean isNewRightPredAlwaysTrue = newRightPredicate.isAlwaysTrue();
+    boolean genPredOnLeft = join.getJoinType() == JoinRelType.RIGHT || join.getJoinType() == JoinRelType.INNER;
+    boolean genPredOnRight = join.getJoinType() == JoinRelType.LEFT || join.getJoinType() == JoinRelType.INNER;
 
-    if (isNewLeftPredicateAlwaysTrue && isNewRightPredAlwaysTrue) {
+    RexNode newLeftPredicate = getNewPredicate(join, registry, joinPredInfo, leftPushedPredicates, genPredOnLeft, 0);
+    RexNode newRightPredicate = getNewPredicate(join, registry, joinPredInfo, rightPushedPredicates, genPredOnRight, 1);
+
+    if (newLeftPredicate.isAlwaysTrue() && newRightPredicate.isAlwaysTrue()) {
       return;
     }
 
-    if (!isNewLeftPredicateAlwaysTrue) {
-      RelNode curr = lChild;
-      lChild = filterFactory.createFilter(lChild, newLeftPredicate);
-      call.getPlanner().onCopy(curr, lChild);
-    }
-    if (!isNewRightPredAlwaysTrue) {
-      RelNode curr = rChild;
-      rChild = filterFactory.createFilter(rChild, newRightPredicate);
-      call.getPlanner().onCopy(curr, rChild);
-    }
+    RelNode lChild = getNewChild(call, join, join.getLeft(), newLeftPredicate);
+    RelNode rChild = getNewChild(call, join, join.getRight(), newRightPredicate);
 
-    Join newJoin = join.copy(join.getTraitSet(), join.getCondition(),
-                             lChild, rChild, join.getJoinType(), join.isSemiJoinDone());
+    Join newJoin = join.copy(join.getTraitSet(), join.getCondition(), lChild, rChild, join.getJoinType(),
+        join.isSemiJoinDone());
     call.getPlanner().onCopy(join, newJoin);
 
     // Register information about created predicates
@@ -166,10 +115,31 @@ public final class HiveJoinAddNotNullRule extends RelOptRule {
     call.transformTo(newJoin);
   }
 
-  private static List<RexNode> getNotNullConditions(RelOptCluster cluster,
-                                                    RexBuilder rexBuilder, List<RexNode> inputJoinExprs,
-                                                    Set<String> pushedPredicates) {
-    final List<RexNode> newConditions = Lists.newArrayList();
+  private RexNode getNewPredicate(Join join, HiveRulesRegistry registry, JoinPredicateInfo joinPredInfo,
+      Set<String> pushedPredicates, boolean genPred, int pos) {
+    RexBuilder rexBuilder = join.getCluster().getRexBuilder();
+
+    if (genPred) {
+      List<RexNode> joinExprsList = new ArrayList<>();
+      for (JoinLeafPredicateInfo joinLeafPredicateInfo : joinPredInfo.getEquiJoinPredicateElements()) {
+        joinExprsList.addAll(joinLeafPredicateInfo.getJoinExprs(pos));
+      }
+      for (JoinLeafPredicateInfo joinLeafPredicateInfo : joinPredInfo.getNonEquiJoinPredicateElements()) {
+        if (SqlKind.COMPARISON.contains(joinLeafPredicateInfo.getComparisonType())) {
+          joinExprsList.addAll(joinLeafPredicateInfo.getJoinExprs(pos));
+        }
+      }
+
+      List<RexNode> newConditions = getNotNullConditions(rexBuilder, joinExprsList, pushedPredicates);
+      return RexUtil.composeConjunction(rexBuilder, newConditions, false);
+    } else {
+      return rexBuilder.makeLiteral(true);
+    }
+  }
+
+  private static List<RexNode> getNotNullConditions(RexBuilder rexBuilder, List<RexNode> inputJoinExprs,
+      Set<String> pushedPredicates) {
+    List<RexNode> newConditions = Lists.newArrayList();
 
     for (RexNode rexNode : inputJoinExprs) {
       RexNode cond = rexBuilder.makeCall(SqlStdOperatorTable.IS_NOT_NULL, rexNode);
@@ -180,4 +150,14 @@ public final class HiveJoinAddNotNullRule extends RelOptRule {
     }
     return newConditions;
   }
+
+  private RelNode getNewChild(RelOptRuleCall call, Join join, RelNode child, RexNode newPredicate) {
+    if (!newPredicate.isAlwaysTrue()) {
+      RelNode newChild = filterFactory.createFilter(child, newPredicate);
+      call.getPlanner().onCopy(child, newChild);
+      return newChild;
+    }
+
+    return child;
+  }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRulesRegistry.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRulesRegistry.java
index 261507a..6506dc5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRulesRegistry.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRulesRegistry.java
@@ -31,7 +31,7 @@ import com.google.common.collect.Sets;
 public class HiveRulesRegistry {
 
   private SetMultimap<RelOptRule, RelNode> registryVisited;
-  private ListMultimap<RelNode,Set<String>> registryPushedPredicates;
+  private ListMultimap<RelNode, Set<String>> registryPushedPredicates;
 
   public HiveRulesRegistry() {
     this.registryVisited = HashMultimap.create();
diff --git a/ql/src/test/queries/clientpositive/join_by_range_rule_not_null.q b/ql/src/test/queries/clientpositive/join_by_range_rule_not_null.q
new file mode 100644
index 0000000..abbebb7
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/join_by_range_rule_not_null.q
@@ -0,0 +1,9 @@
+--! qt:dataset:src1
+--! qt:dataset:src
+
+EXPLAIN SELECT * FROM src a JOIN src1 b ON a.key = b.key;
+EXPLAIN SELECT * FROM src a JOIN src1 b ON a.key < b.key;
+EXPLAIN SELECT * FROM src a JOIN src1 b ON a.key = b.key AND a.value >= b.value;
+EXPLAIN SELECT * FROM src a JOIN src1 b ON a.key > b.value;
+EXPLAIN SELECT * FROM src a JOIN src1 b ON a.key > b.key OR 1 = 1;
+EXPLAIN SELECT * FROM src a JOIN src1 b ON a.key IS DISTINCT FROM b.key;
diff --git a/ql/src/test/results/clientnegative/subquery_scalar_multi_rows.q.out b/ql/src/test/results/clientnegative/subquery_scalar_multi_rows.q.out
index 92f1365..7ca260c 100644
--- a/ql/src/test/results/clientnegative/subquery_scalar_multi_rows.q.out
+++ b/ql/src/test/results/clientnegative/subquery_scalar_multi_rows.q.out
@@ -1,5 +1,5 @@
-Warning: Shuffle Join JOIN[17][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product
-Warning: Shuffle Join JOIN[14][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[19][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[16][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: select p_name from part where p_size > (select p_size from part)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
diff --git a/ql/src/test/results/clientpositive/interval_3.q.out b/ql/src/test/results/clientpositive/interval_3.q.out
index a549751..ba146df 100644
--- a/ql/src/test/results/clientpositive/interval_3.q.out
+++ b/ql/src/test/results/clientpositive/interval_3.q.out
@@ -108,7 +108,7 @@ POSTHOOK: query: create table date_dim_d1(
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@date_dim_d1
-Warning: Shuffle Join JOIN[6][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[8][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: EXPLAIN SELECT
            d1.d_week_seq
     FROM
@@ -153,26 +153,34 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: d1
+            filterExpr: CAST( d_date AS DATE) is not null (type: boolean)
             Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-            Select Operator
-              expressions: d_week_seq (type: int), ((((((CAST( d_date AS DATE) + INTERVAL'1-0') + INTERVAL'0-2') + INTERVAL'5 00:00:00.000000000') + INTERVAL'0 04:00:00.000000000') + INTERVAL'0 00:10:00.000000000') + INTERVAL'0 00:00:09.000000000') (type: timestamp), (CAST( d_date AS DATE) + INTERVAL'1-2') (type: date)
-              outputColumnNames: _col0, _col1, _col2
+            Filter Operator
+              predicate: CAST( d_date AS DATE) is not null (type: boolean)
               Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-              Reduce Output Operator
-                sort order: 
+              Select Operator
+                expressions: d_week_seq (type: int), ((((((CAST( d_date AS DATE) + INTERVAL'1-0') + INTERVAL'0-2') + INTERVAL'5 00:00:00.000000000') + INTERVAL'0 04:00:00.000000000') + INTERVAL'0 00:10:00.000000000') + INTERVAL'0 00:00:09.000000000') (type: timestamp), (CAST( d_date AS DATE) + INTERVAL'1-2') (type: date)
+                outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                value expressions: _col0 (type: int), _col1 (type: timestamp), _col2 (type: date)
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col0 (type: int), _col1 (type: timestamp), _col2 (type: date)
           TableScan
             alias: d3
+            filterExpr: (CAST( CAST( d_date AS DATE) AS TIMESTAMP) is not null and CAST( d_date AS DATE) is not null) (type: boolean)
             Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE
-            Select Operator
-              expressions: CAST( CAST( d_date AS DATE) AS TIMESTAMP) (type: timestamp), CAST( d_date AS DATE) (type: date)
-              outputColumnNames: _col0, _col1
+            Filter Operator
+              predicate: (CAST( CAST( d_date AS DATE) AS TIMESTAMP) is not null and CAST( d_date AS DATE) is not null) (type: boolean)
               Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE
-              Reduce Output Operator
-                sort order: 
+              Select Operator
+                expressions: CAST( CAST( d_date AS DATE) AS TIMESTAMP) (type: timestamp), CAST( d_date AS DATE) (type: date)
+                outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE
-                value expressions: _col0 (type: timestamp), _col1 (type: date)
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col0 (type: timestamp), _col1 (type: date)
       Reduce Operator Tree:
         Join Operator
           condition map:
diff --git a/ql/src/test/results/clientpositive/join43.q.out b/ql/src/test/results/clientpositive/join43.q.out
index 7a40f0b..08e5c91 100644
--- a/ql/src/test/results/clientpositive/join43.q.out
+++ b/ql/src/test/results/clientpositive/join43.q.out
@@ -260,10 +260,10 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: cart_history
-            filterExpr: s is not null (type: boolean)
+            filterExpr: (s is not null and time is not null) (type: boolean)
             Statistics: Num rows: 6 Data size: 534 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: s is not null (type: boolean)
+              predicate: (s is not null and time is not null) (type: boolean)
               Statistics: Num rows: 6 Data size: 534 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: s (type: string), time (type: int)
@@ -277,10 +277,10 @@ STAGE PLANS:
                   value expressions: _col1 (type: int)
           TableScan
             alias: purchase_history
-            filterExpr: s is not null (type: boolean)
+            filterExpr: (s is not null and time is not null) (type: boolean)
             Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: s is not null (type: boolean)
+              predicate: (s is not null and time is not null) (type: boolean)
               Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: s (type: string), time (type: int)
@@ -495,10 +495,10 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: cart_history
-            filterExpr: s is not null (type: boolean)
+            filterExpr: (s is not null and time is not null) (type: boolean)
             Statistics: Num rows: 6 Data size: 534 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: s is not null (type: boolean)
+              predicate: (s is not null and time is not null) (type: boolean)
               Statistics: Num rows: 6 Data size: 534 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: s (type: string), time (type: int)
@@ -512,10 +512,10 @@ STAGE PLANS:
                   value expressions: _col1 (type: int)
           TableScan
             alias: purchase_history
-            filterExpr: s is not null (type: boolean)
+            filterExpr: (s is not null and time is not null) (type: boolean)
             Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: s is not null (type: boolean)
+              predicate: (s is not null and time is not null) (type: boolean)
               Statistics: Num rows: 4 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: s (type: string), time (type: int)
diff --git a/ql/src/test/results/clientpositive/join_by_range_rule_not_null.q.out b/ql/src/test/results/clientpositive/join_by_range_rule_not_null.q.out
new file mode 100644
index 0000000..04e38e8
--- /dev/null
+++ b/ql/src/test/results/clientpositive/join_by_range_rule_not_null.q.out
@@ -0,0 +1,430 @@
+PREHOOK: query: EXPLAIN SELECT * FROM src a JOIN src1 b ON a.key = b.key
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Input: default@src1
+#### A masked pattern was here ####
+POSTHOOK: query: EXPLAIN SELECT * FROM src a JOIN src1 b ON a.key = b.key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Input: default@src1
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: a
+            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)
+                  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)
+          TableScan
+            alias: b
+            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
+                Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  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)
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Inner Join 0 to 1
+          keys:
+            0 _col0 (type: string)
+            1 _col0 (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3
+          Statistics: Num rows: 39 Data size: 13767 Basic stats: COMPLETE Column stats: COMPLETE
+          File Output Operator
+            compressed: false
+            Statistics: Num rows: 39 Data size: 13767 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
+      limit: -1
+      Processor Tree:
+        ListSink
+
+Warning: Shuffle Join JOIN[8][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+PREHOOK: query: EXPLAIN SELECT * FROM src a JOIN src1 b ON a.key < b.key
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Input: default@src1
+#### A masked pattern was here ####
+POSTHOOK: query: EXPLAIN SELECT * FROM src a JOIN src1 b ON a.key < b.key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Input: default@src1
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: a
+            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
+                  sort order: 
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: string), _col1 (type: string)
+          TableScan
+            alias: b
+            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
+                Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: string), _col1 (type: string)
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Inner Join 0 to 1
+          keys:
+            0 
+            1 
+          outputColumnNames: _col0, _col1, _col2, _col3
+          Statistics: Num rows: 12500 Data size: 4412500 Basic stats: COMPLETE Column stats: COMPLETE
+          Filter Operator
+            predicate: (_col0 < _col2) (type: boolean)
+            Statistics: Num rows: 4166 Data size: 1470598 Basic stats: COMPLETE Column stats: COMPLETE
+            File Output Operator
+              compressed: false
+              Statistics: Num rows: 4166 Data size: 1470598 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
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: EXPLAIN SELECT * FROM src a JOIN src1 b ON a.key = b.key AND a.value >= b.value
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Input: default@src1
+#### A masked pattern was here ####
+POSTHOOK: query: EXPLAIN SELECT * FROM src a JOIN src1 b ON a.key = b.key AND a.value >= b.value
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Input: default@src1
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: a
+            filterExpr: (key is not null and 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)
+              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)
+                  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)
+          TableScan
+            alias: b
+            filterExpr: (key is not null and value 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 and value 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
+                Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  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)
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Inner Join 0 to 1
+          keys:
+            0 _col0 (type: string)
+            1 _col0 (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3
+          Statistics: Num rows: 39 Data size: 13767 Basic stats: COMPLETE Column stats: COMPLETE
+          Filter Operator
+            predicate: (_col1 >= _col3) (type: boolean)
+            Statistics: Num rows: 13 Data size: 4589 Basic stats: COMPLETE Column stats: COMPLETE
+            File Output Operator
+              compressed: false
+              Statistics: Num rows: 13 Data size: 4589 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
+      limit: -1
+      Processor Tree:
+        ListSink
+
+Warning: Shuffle Join JOIN[8][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+PREHOOK: query: EXPLAIN SELECT * FROM src a JOIN src1 b ON a.key > b.value
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Input: default@src1
+#### A masked pattern was here ####
+POSTHOOK: query: EXPLAIN SELECT * FROM src a JOIN src1 b ON a.key > b.value
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Input: default@src1
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: a
+            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
+                  sort order: 
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: string), _col1 (type: string)
+          TableScan
+            alias: b
+            filterExpr: value is not null (type: boolean)
+            Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
+            Filter Operator
+              predicate: value 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
+                Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: string), _col1 (type: string)
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Inner Join 0 to 1
+          keys:
+            0 
+            1 
+          outputColumnNames: _col0, _col1, _col2, _col3
+          Statistics: Num rows: 12500 Data size: 4412500 Basic stats: COMPLETE Column stats: COMPLETE
+          Filter Operator
+            predicate: (_col0 > _col3) (type: boolean)
+            Statistics: Num rows: 4166 Data size: 1470598 Basic stats: COMPLETE Column stats: COMPLETE
+            File Output Operator
+              compressed: false
+              Statistics: Num rows: 4166 Data size: 1470598 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
+      limit: -1
+      Processor Tree:
+        ListSink
+
+Warning: Shuffle Join JOIN[6][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+PREHOOK: query: EXPLAIN SELECT * FROM src a JOIN src1 b ON a.key > b.key OR 1 = 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Input: default@src1
+#### A masked pattern was here ####
+POSTHOOK: query: EXPLAIN SELECT * FROM src a JOIN src1 b ON a.key > b.key OR 1 = 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Input: default@src1
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: a
+            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
+                sort order: 
+                Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                value expressions: _col0 (type: string), _col1 (type: string)
+          TableScan
+            alias: b
+            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
+              Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
+              Reduce Output Operator
+                sort order: 
+                Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
+                value expressions: _col0 (type: string), _col1 (type: string)
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Inner Join 0 to 1
+          keys:
+            0 
+            1 
+          outputColumnNames: _col0, _col1, _col2, _col3
+          Statistics: Num rows: 12500 Data size: 4412500 Basic stats: COMPLETE Column stats: COMPLETE
+          File Output Operator
+            compressed: false
+            Statistics: Num rows: 12500 Data size: 4412500 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
+      limit: -1
+      Processor Tree:
+        ListSink
+
+Warning: Shuffle Join JOIN[4][tables = [a, b]] in Stage 'Stage-1:MAPRED' is a cross product
+PREHOOK: query: EXPLAIN SELECT * FROM src a JOIN src1 b ON a.key IS DISTINCT FROM b.key
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Input: default@src1
+#### A masked pattern was here ####
+POSTHOOK: query: EXPLAIN SELECT * FROM src a JOIN src1 b ON a.key IS DISTINCT FROM b.key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Input: default@src1
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: a
+            Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+            Reduce Output Operator
+              sort order: 
+              Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+              value expressions: key (type: string), value (type: string)
+          TableScan
+            alias: b
+            Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
+            Reduce Output Operator
+              sort order: 
+              Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
+              value expressions: key (type: string), value (type: string)
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Inner Join 0 to 1
+          keys:
+            0 
+            1 
+          outputColumnNames: _col0, _col1, _col5, _col6
+          Statistics: Num rows: 12500 Data size: 4412500 Basic stats: COMPLETE Column stats: COMPLETE
+          Filter Operator
+            predicate: (_col0 IS DISTINCT FROM _col5) (type: boolean)
+            Statistics: Num rows: 12500 Data size: 4412500 Basic stats: COMPLETE Column stats: COMPLETE
+            Select Operator
+              expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string)
+              outputColumnNames: _col0, _col1, _col2, _col3
+              Statistics: Num rows: 12500 Data size: 4412500 Basic stats: COMPLETE Column stats: COMPLETE
+              File Output Operator
+                compressed: false
+                Statistics: Num rows: 12500 Data size: 4412500 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
+      limit: -1
+      Processor Tree:
+        ListSink
+
diff --git a/ql/src/test/results/clientpositive/join_merging.q.out b/ql/src/test/results/clientpositive/join_merging.q.out
index 2688f83..af840a8 100644
--- a/ql/src/test/results/clientpositive/join_merging.q.out
+++ b/ql/src/test/results/clientpositive/join_merging.q.out
@@ -149,10 +149,10 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: p2
-            filterExpr: p_partkey is not null (type: boolean)
+            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 (type: boolean)
+              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
               Select Operator
                 expressions: p_partkey (type: int), p_size (type: int), (p_size + 10) (type: int)
diff --git a/ql/src/test/results/clientpositive/llap/cross_prod_1.q.out b/ql/src/test/results/clientpositive/llap/cross_prod_1.q.out
index d289837..e34a6b4 100644
--- a/ql/src/test/results/clientpositive/llap/cross_prod_1.q.out
+++ b/ql/src/test/results/clientpositive/llap/cross_prod_1.q.out
@@ -210,7 +210,7 @@ POSTHOOK: Input: default@x_n0
 114	val_114	111	val_111
 114	val_114	113	val_113
 114	val_114	114	val_114
-Warning: Shuffle Join MERGEJOIN[13][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+Warning: Shuffle Join MERGEJOIN[15][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
 PREHOOK: query: explain select * from X_n0 as A join X_n0 as B on A.key<B.key
 PREHOOK: type: QUERY
 PREHOOK: Input: default@x_n0
@@ -235,30 +235,38 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: a
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 10 Data size: 1780 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: string), value (type: string)
-                    outputColumnNames: _col0, _col1
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 10 Data size: 1780 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      sort order: 
+                    Select Operator
+                      expressions: key (type: string), value (type: string)
+                      outputColumnNames: _col0, _col1
                       Statistics: Num rows: 10 Data size: 1780 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: _col0 (type: string), _col1 (type: string)
+                      Reduce Output Operator
+                        sort order: 
+                        Statistics: Num rows: 10 Data size: 1780 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: b
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 10 Data size: 1780 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: string), value (type: string)
-                    outputColumnNames: _col0, _col1
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 10 Data size: 1780 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      sort order: 
+                    Select Operator
+                      expressions: key (type: string), value (type: string)
+                      outputColumnNames: _col0, _col1
                       Statistics: Num rows: 10 Data size: 1780 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: _col0 (type: string), _col1 (type: string)
+                      Reduce Output Operator
+                        sort order: 
+                        Statistics: Num rows: 10 Data size: 1780 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: string), _col1 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
@@ -287,7 +295,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[15][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+Warning: Shuffle Join MERGEJOIN[17][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
 PREHOOK: query: select * from X_n0 as A join X_n0 as B on A.key<B.key order by A.key, B.key
 PREHOOK: type: QUERY
 PREHOOK: Input: default@x_n0
diff --git a/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction.q.out b/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction.q.out
index 8ea8f53..ba8e5ee 100644
--- a/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction.q.out
+++ b/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction.q.out
@@ -4226,10 +4226,10 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: srcpart_date_n7
-                  filterExpr: key is not null (type: boolean)
+                  filterExpr: (key is not null and value is not null) (type: boolean)
                   Statistics: Num rows: 2000 Data size: 356000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: key is not null (type: boolean)
+                    predicate: (key is not null and value is not null) (type: boolean)
                     Statistics: Num rows: 2000 Data size: 356000 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: string), value (type: string)
@@ -4264,10 +4264,10 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: srcpart_small_n3
-                  filterExpr: key1 is not null (type: boolean)
+                  filterExpr: (key1 is not null and value1 is not null) (type: boolean)
                   Statistics: Num rows: 20 Data size: 3560 Basic stats: PARTIAL Column stats: PARTIAL
                   Filter Operator
-                    predicate: key1 is not null (type: boolean)
+                    predicate: (key1 is not null and value1 is not null) (type: boolean)
                     Statistics: Num rows: 20 Data size: 3560 Basic stats: PARTIAL Column stats: PARTIAL
                     Select Operator
                       expressions: key1 (type: string), value1 (type: string)
diff --git a/ql/src/test/results/clientpositive/llap/groupby_groupingset_bug.q.out b/ql/src/test/results/clientpositive/llap/groupby_groupingset_bug.q.out
index 5601028..7e7a7f7 100644
--- a/ql/src/test/results/clientpositive/llap/groupby_groupingset_bug.q.out
+++ b/ql/src/test/results/clientpositive/llap/groupby_groupingset_bug.q.out
@@ -226,91 +226,93 @@ Stage-0
     limit:100
     Stage-1
       Reducer 6 vectorized, llap
-      File Output Operator [FS_141]
-        Limit [LIM_140] (rows=2 width=8)
+      File Output Operator [FS_143]
+        Limit [LIM_142] (rows=2 width=8)
           Number of rows:100
-          Select Operator [SEL_139] (rows=2 width=8)
+          Select Operator [SEL_141] (rows=2 width=8)
             Output:["_col0"]
           <-Reducer 5 [SIMPLE_EDGE] vectorized, llap
-            SHUFFLE [RS_138]
-              Select Operator [SEL_137] (rows=2 width=8)
+            SHUFFLE [RS_140]
+              Select Operator [SEL_139] (rows=2 width=8)
                 Output:["_col0"]
-                Group By Operator [GBY_136] (rows=2 width=12)
+                Group By Operator [GBY_138] (rows=2 width=12)
                   Output:["_col0","_col1"],aggregations:["count(VALUE._col0)"],keys:KEY._col0
                 <-Map 4 [SIMPLE_EDGE] vectorized, llap
-                  SHUFFLE [RS_135]
+                  SHUFFLE [RS_137]
                     PartitionCols:_col0
-                    Group By Operator [GBY_134] (rows=2 width=12)
+                    Group By Operator [GBY_136] (rows=2 width=12)
                       Output:["_col0","_col1"],aggregations:["count()"],keys:_col8
-                      Map Join Operator [MAPJOIN_133] (rows=5185194 width=4)
-                        Conds:MAPJOIN_132._col6=RS_126._col0(Inner),Output:["_col8"]
+                      Map Join Operator [MAPJOIN_135] (rows=5185194 width=4)
+                        Conds:MAPJOIN_134._col6=RS_128._col0(Inner),Output:["_col8"]
                       <-Map 7 [BROADCAST_EDGE] vectorized, llap
-                        BROADCAST [RS_126]
+                        BROADCAST [RS_128]
                           PartitionCols:_col0
-                          Map Join Operator [MAPJOIN_125] (rows=28 width=8)
-                            Conds:SEL_124._col1=RS_122._col0(Inner),Output:["_col0","_col1"]
+                          Map Join Operator [MAPJOIN_127] (rows=28 width=8)
+                            Conds:SEL_126._col1=RS_124._col0(Inner),Output:["_col0","_col1"]
                           <-Reducer 9 [BROADCAST_EDGE] vectorized, llap
-                            BROADCAST [RS_122]
+                            BROADCAST [RS_124]
                               PartitionCols:_col0
-                              Group By Operator [GBY_121] (rows=2 width=4)
+                              Group By Operator [GBY_123] (rows=2 width=4)
                                 Output:["_col0"],keys:KEY._col0
                               <-Map 8 [SIMPLE_EDGE] vectorized, llap
-                                SHUFFLE [RS_120]
+                                SHUFFLE [RS_122]
                                   PartitionCols:_col0
-                                  Group By Operator [GBY_119] (rows=2 width=4)
+                                  Group By Operator [GBY_121] (rows=2 width=4)
                                     Output:["_col0"],keys:d_month_seq
-                                    Select Operator [SEL_118] (rows=4 width=12)
+                                    Select Operator [SEL_120] (rows=4 width=12)
                                       Output:["d_month_seq"]
-                                      Filter Operator [FIL_117] (rows=4 width=12)
+                                      Filter Operator [FIL_119] (rows=4 width=12)
                                         predicate:(((d_year * d_moy) > 200000) and (d_moy = 2) and (d_year = 2000) and d_month_seq is not null)
-                                        TableScan [TS_16] (rows=28 width=12)
+                                        TableScan [TS_17] (rows=28 width=12)
                                           default@x1_date_dim,x1_date_dim,Tbl:COMPLETE,Col:COMPLETE,Output:["d_month_seq","d_year","d_moy"]
-                          <-Select Operator [SEL_124] (rows=28 width=8)
+                          <-Select Operator [SEL_126] (rows=28 width=8)
                               Output:["_col0","_col1"]
-                              Filter Operator [FIL_123] (rows=28 width=8)
+                              Filter Operator [FIL_125] (rows=28 width=8)
                                 predicate:(d_date_sk is not null and d_month_seq is not null)
-                                TableScan [TS_13] (rows=28 width=8)
+                                TableScan [TS_14] (rows=28 width=8)
                                   default@x1_date_dim,d,Tbl:COMPLETE,Col:COMPLETE,Output:["d_date_sk","d_month_seq"]
-                        Dynamic Partitioning Event Operator [EVENT_129] (rows=1 width=4)
-                          Group By Operator [GBY_128] (rows=1 width=4)
+                        Dynamic Partitioning Event Operator [EVENT_131] (rows=1 width=4)
+                          Group By Operator [GBY_130] (rows=1 width=4)
                             Output:["_col0"],keys:_col0
-                            Select Operator [SEL_127] (rows=28 width=8)
+                            Select Operator [SEL_129] (rows=28 width=8)
                               Output:["_col0"]
-                               Please refer to the previous Map Join Operator [MAPJOIN_125]
-                      <-Map Join Operator [MAPJOIN_132] (rows=370371 width=4)
-                          Conds:RS_30._col0=SEL_131._col0(Inner),Output:["_col6"]
+                               Please refer to the previous Map Join Operator [MAPJOIN_127]
+                      <-Map Join Operator [MAPJOIN_134] (rows=370371 width=4)
+                          Conds:RS_31._col0=SEL_133._col0(Inner),Output:["_col6"]
                         <-Map 1 [BROADCAST_EDGE] llap
-                          BROADCAST [RS_30]
+                          BROADCAST [RS_31]
                             PartitionCols:_col0
-                            Map Join Operator [MAPJOIN_101] (rows=6 width=228)
-                              Conds:SEL_2._col1=RS_114._col1(Inner),Output:["_col0","_col2","_col3"],residual filter predicates:{(_col2 > _col3)}
+                            Map Join Operator [MAPJOIN_102] (rows=6 width=228)
+                              Conds:SEL_2._col1=RS_116._col1(Inner),Output:["_col0","_col2","_col3"],residual filter predicates:{(_col2 > _col3)}
                             <-Reducer 3 [BROADCAST_EDGE] vectorized, llap
-                              BROADCAST [RS_114]
+                              BROADCAST [RS_116]
                                 PartitionCols:_col1
-                                Select Operator [SEL_113] (rows=1 width=197)
+                                Select Operator [SEL_115] (rows=1 width=197)
                                   Output:["_col0","_col1"]
-                                  Group By Operator [GBY_112] (rows=1 width=197)
-                                    Output:["_col0","_col1"],aggregations:["min(VALUE._col0)"],keys:KEY._col0
-                                  <-Map 2 [SIMPLE_EDGE] vectorized, llap
-                                    SHUFFLE [RS_111]
-                                      PartitionCols:_col0
-                                      Group By Operator [GBY_110] (rows=1 width=197)
-                                        Output:["_col0","_col1"],aggregations:["min(i_current_price)"],keys:i_category
-                                        Filter Operator [FIL_109] (rows=18 width=197)
-                                          predicate:i_category is not null
-                                          TableScan [TS_3] (rows=18 width=197)
-                                            default@x1_item,j,Tbl:COMPLETE,Col:COMPLETE,Output:["i_category","i_current_price"]
+                                  Filter Operator [FIL_114] (rows=1 width=197)
+                                    predicate:_col1 is not null
+                                    Group By Operator [GBY_113] (rows=1 width=197)
+                                      Output:["_col0","_col1"],aggregations:["min(VALUE._col0)"],keys:KEY._col0
+                                    <-Map 2 [SIMPLE_EDGE] vectorized, llap
+                                      SHUFFLE [RS_112]
+                                        PartitionCols:_col0
+                                        Group By Operator [GBY_111] (rows=1 width=197)
+                                          Output:["_col0","_col1"],aggregations:["min(i_current_price)"],keys:i_category
+                                          Filter Operator [FIL_110] (rows=18 width=197)
+                                            predicate:i_category is not null
+                                            TableScan [TS_3] (rows=18 width=197)
+                                              default@x1_item,j,Tbl:COMPLETE,Col:COMPLETE,Output:["i_category","i_current_price"]
                             <-Select Operator [SEL_2] (rows=18 width=201)
                                 Output:["_col0","_col1","_col2"]
-                                Filter Operator [FIL_57] (rows=18 width=201)
-                                  predicate:(i_category is not null and i_item_sk is not null)
+                                Filter Operator [FIL_58] (rows=18 width=201)
+                                  predicate:(i_category is not null and i_current_price is not null and i_item_sk is not null)
                                   TableScan [TS_0] (rows=18 width=201)
                                     default@x1_item,i,Tbl:COMPLETE,Col:COMPLETE,Output:["i_item_sk","i_category","i_current_price"]
-                        <-Select Operator [SEL_131] (rows=123457 width=8)
+                        <-Select Operator [SEL_133] (rows=123457 width=8)
                             Output:["_col0","_col1"]
-                            Filter Operator [FIL_130] (rows=123457 width=8)
+                            Filter Operator [FIL_132] (rows=123457 width=8)
                               predicate:ss_item_sk is not null
-                              TableScan [TS_10] (rows=123457 width=8)
+                              TableScan [TS_11] (rows=123457 width=8)
                                 default@x1_store_sales,s,Tbl:COMPLETE,Col:COMPLETE,Output:["ss_item_sk"]
 
 PREHOOK: query: select   count(*) cnt
diff --git a/ql/src/test/results/clientpositive/llap/semijoin.q.out b/ql/src/test/results/clientpositive/llap/semijoin.q.out
index 985eeb7..ca7a33d 100644
--- a/ql/src/test/results/clientpositive/llap/semijoin.q.out
+++ b/ql/src/test/results/clientpositive/llap/semijoin.q.out
@@ -3148,7 +3148,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: outr
-                  filterExpr: ((key is not null and value is not null) or key is not null) (type: boolean)
+                  filterExpr: (key is not null and 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)
@@ -3162,13 +3162,6 @@ 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 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
                         sort order: 
                         Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/subquery_corr.q.out b/ql/src/test/results/clientpositive/llap/subquery_corr.q.out
index f6d692a..6c140ab 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_corr.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_corr.q.out
@@ -22,10 +22,10 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: b
-                  filterExpr: key is not null (type: boolean)
+                  filterExpr: (key is not null and 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 (type: boolean)
+                    predicate: (key is not null and value 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)
@@ -128,10 +128,10 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: b
-                  filterExpr: key is not null (type: boolean)
+                  filterExpr: (key is not null and 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 (type: boolean)
+                    predicate: (key is not null and value 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)
@@ -734,10 +734,10 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: b
-                  filterExpr: key is not null (type: boolean)
+                  filterExpr: (key is not null and 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 (type: boolean)
+                    predicate: (key is not null and value 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)
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 84611ce..cd31afb 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_in.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_in.q.out
@@ -957,10 +957,10 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: b
-                  filterExpr: (key is not null or (value is not null and key is not null)) (type: boolean)
+                  filterExpr: (key is not null and 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 (type: boolean)
+                    predicate: (key is not null and value 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)
@@ -972,9 +972,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)
-                  Filter Operator
-                    predicate: (key is not null and value is not null) (type: boolean)
-                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
                       keys: key (type: string), value (type: string)
                       minReductionHashAggr: 0.0
@@ -4696,7 +4693,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[52][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 4' 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
@@ -4745,15 +4742,19 @@ STAGE PLANS:
             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
-                  Select Operator
-                    expressions: p_partkey (type: int), p_size (type: int)
-                    outputColumnNames: _col0, _col1
+                  Filter Operator
+                    predicate: p_partkey is not null (type: boolean)
                     Statistics: Num rows: 26 Data size: 208 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      sort order: 
+                    Select Operator
+                      expressions: p_partkey (type: int), p_size (type: int)
+                      outputColumnNames: _col0, _col1
                       Statistics: Num rows: 26 Data size: 208 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: _col0 (type: int), _col1 (type: int)
+                      Reduce Output Operator
+                        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 
@@ -4867,7 +4868,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[52][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 4' 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
diff --git a/ql/src/test/results/clientpositive/llap/subquery_notin.q.out b/ql/src/test/results/clientpositive/llap/subquery_notin.q.out
index e55396f..c92dd35 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_notin.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_notin.q.out
@@ -7353,7 +7353,7 @@ STAGE PLANS:
                       Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                       value expressions: _col0 (type: string)
                   Filter Operator
-                    predicate: (key > '9') (type: boolean)
+                    predicate: ((key > '9') 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)
diff --git a/ql/src/test/results/clientpositive/llap/subquery_scalar.q.out b/ql/src/test/results/clientpositive/llap/subquery_scalar.q.out
index 3bd44d6..0d8ff14 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_scalar.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_scalar.q.out
@@ -79,7 +79,7 @@ POSTHOOK: Lineage: part_null_n0.p_partkey SCRIPT []
 POSTHOOK: Lineage: part_null_n0.p_retailprice SCRIPT []
 POSTHOOK: Lineage: part_null_n0.p_size SCRIPT []
 POSTHOOK: Lineage: part_null_n0.p_type SCRIPT []
-Warning: Shuffle Join MERGEJOIN[17][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+Warning: Shuffle Join MERGEJOIN[20][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
 PREHOOK: query: explain select * from part where p_size > (select avg(p_size) from part_null_n0)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
@@ -107,15 +107,19 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: part
+                  filterExpr: UDFToDouble(p_size) is not null (type: boolean)
                   Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), UDFToDouble(p_size) (type: double)
-                    outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
-                    Statistics: Num rows: 26 Data size: 16302 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      sort order: 
+                  Filter Operator
+                    predicate: UDFToDouble(p_size) is not null (type: boolean)
+                    Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), UDFToDouble(p_size) (type: double)
+                      outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
                       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), _col9 (type: double)
+                      Reduce Output Operator
+                        sort order: 
+                        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), _col9 (type: double)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 3 
@@ -170,14 +174,17 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: NONE
-                Select Operator
-                  expressions: (UDFToDouble(_col0) / _col1) (type: double)
-                  outputColumnNames: _col0
+                Filter Operator
+                  predicate: (UDFToDouble(_col0) is not null and _col1 is not null) (type: boolean)
                   Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    sort order: 
+                  Select Operator
+                    expressions: (UDFToDouble(_col0) / _col1) (type: double)
+                    outputColumnNames: _col0
                     Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col0 (type: double)
+                    Reduce Output Operator
+                      sort order: 
+                      Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col0 (type: double)
 
   Stage: Stage-0
     Fetch Operator
@@ -185,7 +192,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[17][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+Warning: Shuffle Join MERGEJOIN[20][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
 PREHOOK: query: select * from part where p_size > (select avg(p_size) from part_null_n0)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
@@ -207,8 +214,8 @@ POSTHOOK: Input: default@part_null_n0
 78486	almond azure blanched chiffon midnight	Manufacturer#5	Brand#52	LARGE BRUSHED BRASS	23	MED BAG	1464.48	hely blith
 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 
-Warning: Shuffle Join MERGEJOIN[26][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
-Warning: Shuffle Join MERGEJOIN[27][tables = [$hdt$_1, $hdt$_2, $hdt$_0]] in Stage 'Reducer 4' is a cross product
+Warning: Shuffle Join MERGEJOIN[29][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[30][tables = [$hdt$_1, $hdt$_2, $hdt$_0]] in Stage 'Reducer 4' is a cross product
 PREHOOK: query: select * from part where p_size > (select * from tempty_n0)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
@@ -219,8 +226,8 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@part
 POSTHOOK: Input: default@tempty_n0
 #### A masked pattern was here ####
-Warning: Shuffle Join MERGEJOIN[26][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
-Warning: Shuffle Join MERGEJOIN[27][tables = [$hdt$_1, $hdt$_2, $hdt$_0]] in Stage 'Reducer 4' is a cross product
+Warning: Shuffle Join MERGEJOIN[29][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[30][tables = [$hdt$_1, $hdt$_2, $hdt$_0]] in Stage 'Reducer 4' is a cross product
 PREHOOK: query: explain select * from part where p_size > (select * from tempty_n0)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
@@ -268,30 +275,38 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: tempty_n0
+                  filterExpr: UDFToDouble(c) is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 86 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: UDFToDouble(c) (type: double)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: UDFToDouble(c) is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 86 Basic stats: COMPLETE Column stats: NONE
-                    Reduce Output Operator
-                      sort order: 
+                    Select Operator
+                      expressions: UDFToDouble(c) (type: double)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 86 Basic stats: COMPLETE Column stats: NONE
-                      value expressions: _col0 (type: double)
+                      Reduce Output Operator
+                        sort order: 
+                        Statistics: Num rows: 1 Data size: 86 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col0 (type: double)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 6 
             Map Operator Tree:
                 TableScan
                   alias: part
+                  filterExpr: UDFToDouble(p_size) is not null (type: boolean)
                   Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), UDFToDouble(p_size) (type: double)
-                    outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
-                    Statistics: Num rows: 26 Data size: 16302 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      sort order: 
+                  Filter Operator
+                    predicate: UDFToDouble(p_size) is not null (type: boolean)
+                    Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), UDFToDouble(p_size) (type: double)
+                      outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
                       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), _col9 (type: double)
+                      Reduce Output Operator
+                        sort order: 
+                        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), _col9 (type: double)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
@@ -768,8 +783,8 @@ POSTHOOK: Input: default@part_null_n0
 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[30][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
-Warning: Shuffle Join MERGEJOIN[31][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[35][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+Warning: Shuffle Join MERGEJOIN[36][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: explain select * from part where p_size between (select min(p_size) from part) and (select avg(p_size) from part)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
@@ -797,15 +812,19 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: part
+                  filterExpr: UDFToDouble(p_size) is not null (type: boolean)
                   Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), UDFToDouble(p_size) (type: double)
-                    outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
-                    Statistics: Num rows: 26 Data size: 16302 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      sort order: 
+                  Filter Operator
+                    predicate: UDFToDouble(p_size) is not null (type: boolean)
+                    Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), UDFToDouble(p_size) (type: double)
+                      outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
                       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), _col9 (type: double)
+                      Reduce Output Operator
+                        sort order: 
+                        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), _col9 (type: double)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 4 
@@ -886,14 +905,17 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: UDFToDouble(_col0) (type: double)
-                  outputColumnNames: _col0
-                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    sort order: 
+                Filter Operator
+                  predicate: UDFToDouble(_col0) is not null (type: boolean)
+                  Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: UDFToDouble(_col0) (type: double)
+                    outputColumnNames: _col0
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col0 (type: double)
+                    Reduce Output Operator
+                      sort order: 
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                      value expressions: _col0 (type: double)
         Reducer 6 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
@@ -902,14 +924,17 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: (UDFToDouble(_col0) / _col1) (type: double)
-                  outputColumnNames: _col0
-                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    sort order: 
+                Filter Operator
+                  predicate: (UDFToDouble(_col0) is not null and _col1 is not null) (type: boolean)
+                  Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: (UDFToDouble(_col0) / _col1) (type: double)
+                    outputColumnNames: _col0
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col0 (type: double)
+                    Reduce Output Operator
+                      sort order: 
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                      value expressions: _col0 (type: double)
 
   Stage: Stage-0
     Fetch Operator
@@ -917,8 +942,8 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[30][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
-Warning: Shuffle Join MERGEJOIN[31][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[35][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+Warning: Shuffle Join MERGEJOIN[36][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: select * from part where p_size between (select min(p_size) from part) and (select avg(p_size) from part)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
@@ -942,7 +967,7 @@ POSTHOOK: Input: default@part
 42669	almond antique medium spring khaki	Manufacturer#5	Brand#51	STANDARD BURNISHED TIN	6	MED CAN	1611.66	sits haggl
 49671	almond antique gainsboro frosted violet	Manufacturer#4	Brand#41	SMALL BRUSHED BRASS	10	SM BOX	1620.67	ccounts run quick
 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[21][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+Warning: Shuffle Join MERGEJOIN[24][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
 PREHOOK: query: explain select p_mfgr, p_name, p_size from part 
 where part.p_size > 
   (select first_value(p_size) over(partition by p_mfgr order by p_size) as fv from part order by fv limit 1)
@@ -973,15 +998,19 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: part
+                  filterExpr: p_size is not null (type: boolean)
                   Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: p_name (type: string), p_mfgr (type: string), p_size (type: int)
-                    outputColumnNames: _col0, _col1, _col2
+                  Filter Operator
+                    predicate: p_size is not null (type: boolean)
                     Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      sort order: 
+                    Select Operator
+                      expressions: p_name (type: string), p_mfgr (type: string), p_size (type: int)
+                      outputColumnNames: _col0, _col1, _col2
                       Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: int)
+                      Reduce Output Operator
+                        sort order: 
+                        Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: int)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 3 
@@ -1065,10 +1094,13 @@ STAGE PLANS:
                 Limit
                   Number of rows: 1
                   Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    sort order: 
+                  Filter Operator
+                    predicate: _col0 is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col0 (type: int)
+                    Reduce Output Operator
+                      sort order: 
+                      Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                      value expressions: _col0 (type: int)
 
   Stage: Stage-0
     Fetch Operator
@@ -1076,7 +1108,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[21][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+Warning: Shuffle Join MERGEJOIN[24][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
 PREHOOK: query: select p_mfgr, p_name, p_size from part 
 where part.p_size > 
   (select first_value(p_size) over(partition by p_mfgr order by p_size) as fv from part order by fv limit 1)
@@ -1275,10 +1307,10 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: e
-                  filterExpr: p_name is not null (type: boolean)
+                  filterExpr: (p_name is not null and p_size is not null) (type: boolean)
                   Statistics: Num rows: 26 Data size: 3250 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: p_name is not null (type: boolean)
+                    predicate: (p_name is not null and p_size is not null) (type: boolean)
                     Statistics: Num rows: 26 Data size: 3250 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: p_name (type: string), (p_size + 100) (type: int)
@@ -1364,16 +1396,19 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 13 Data size: 1625 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: _col1 (type: int), _col0 (type: string)
-                  outputColumnNames: _col0, _col1
+                Filter Operator
+                  predicate: _col1 is not null (type: boolean)
                   Statistics: Num rows: 13 Data size: 1625 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    key expressions: _col1 (type: string)
-                    sort order: +
-                    Map-reduce partition columns: _col1 (type: string)
+                  Select Operator
+                    expressions: _col1 (type: int), _col0 (type: string)
+                    outputColumnNames: _col0, _col1
                     Statistics: Num rows: 13 Data size: 1625 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col0 (type: int)
+                    Reduce Output Operator
+                      key expressions: _col1 (type: string)
+                      sort order: +
+                      Map-reduce partition columns: _col1 (type: string)
+                      Statistics: Num rows: 13 Data size: 1625 Basic stats: COMPLETE Column stats: COMPLETE
+                      value expressions: _col0 (type: int)
 
   Stage: Stage-0
     Fetch Operator
@@ -1555,10 +1590,10 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: part
-                  filterExpr: p_type is not null (type: boolean)
+                  filterExpr: (p_type is not null and UDFToDouble(p_size) 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 (type: boolean)
+                    predicate: (UDFToDouble(p_size) is not null and p_type is not null) (type: boolean)
                     Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), UDFToDouble(p_size) (type: double)
@@ -1628,16 +1663,19 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                Select Operator
-                  expressions: (UDFToDouble(_col1) / _col2) (type: double), _col0 (type: string)
-                  outputColumnNames: _col0, _col1
+                Filter Operator
+                  predicate: (UDFToDouble(_col1) is not null and _col2 is not null) (type: boolean)
                   Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col1 (type: string)
-                    sort order: +
-                    Map-reduce partition columns: _col1 (type: string)
+                  Select Operator
+                    expressions: (UDFToDouble(_col1) / _col2) (type: double), _col0 (type: string)
+                    outputColumnNames: _col0, _col1
                     Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col0 (type: double)
+                    Reduce Output Operator
+                      key expressions: _col1 (type: string)
+                      sort order: +
+                      Map-reduce partition columns: _col1 (type: string)
+                      Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col0 (type: double)
 
   Stage: Stage-0
     Fetch Operator
@@ -1656,7 +1694,7 @@ POSTHOOK: Input: default@part
 POSTHOOK: Input: default@part_null_n0
 #### A masked pattern was here ####
 192697	almond antique blue firebrick mint	Manufacturer#5	Brand#52	MEDIUM BURNISHED TIN	31	LG DRUM	1789.69	ickly ir
-Warning: Shuffle Join MERGEJOIN[44][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[47][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: explain select * from part where p_size BETWEEN (select min(p_size) from part_null_n0 where part_null_n0.p_type = part.p_type) AND (select max(p_size) from part_null_n0)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
@@ -1686,10 +1724,10 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: part
-                  filterExpr: p_type is not null (type: boolean)
+                  filterExpr: (p_type 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_type is not null (type: boolean)
+                    predicate: (p_size is not null and p_type is not null) (type: boolean)
                     Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string)
@@ -1796,16 +1834,19 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                Select Operator
-                  expressions: _col1 (type: int), _col0 (type: string)
-                  outputColumnNames: _col0, _col1
+                Filter Operator
+                  predicate: _col1 is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col1 (type: string)
-                    sort order: +
-                    Map-reduce partition columns: _col1 (type: string)
+                  Select Operator
+                    expressions: _col1 (type: int), _col0 (type: string)
+                    outputColumnNames: _col0, _col1
                     Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col0 (type: int)
+                    Reduce Output Operator
+                      key expressions: _col1 (type: string)
+                      sort order: +
+                      Map-reduce partition columns: _col1 (type: string)
+                      Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col0 (type: int)
         Reducer 7 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
@@ -1814,10 +1855,13 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                Reduce Output Operator
-                  sort order: 
+                Filter Operator
+                  predicate: _col0 is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                  value expressions: _col0 (type: int)
+                  Reduce Output Operator
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col0 (type: int)
 
   Stage: Stage-0
     Fetch Operator
@@ -1825,7 +1869,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[44][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[47][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: select * from part where p_size BETWEEN (select min(p_size) from part_null_n0 where part_null_n0.p_type = part.p_type) AND (select max(p_size) from part_null_n0)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
@@ -1862,7 +1906,7 @@ POSTHOOK: Input: default@part_null_n0
 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[44][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[47][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: explain select * from part where p_size >= (select min(p_size) from part_null_n0 where part_null_n0.p_type = part.p_type) AND p_retailprice <= (select max(p_retailprice) from part_null_n0)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
@@ -1892,10 +1936,10 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: part
-                  filterExpr: p_type is not null (type: boolean)
+                  filterExpr: (p_type is not null and p_size is not null and p_retailprice 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 (type: boolean)
+                    predicate: (p_retailprice is not null and p_size is not null and p_type is not null) (type: boolean)
                     Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string)
@@ -2002,16 +2046,19 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                Select Operator
-                  expressions: _col1 (type: int), _col0 (type: string)
-                  outputColumnNames: _col0, _col1
+                Filter Operator
+                  predicate: _col1 is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col1 (type: string)
-                    sort order: +
-                    Map-reduce partition columns: _col1 (type: string)
+                  Select Operator
+                    expressions: _col1 (type: int), _col0 (type: string)
+                    outputColumnNames: _col0, _col1
                     Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col0 (type: int)
+                    Reduce Output Operator
+                      key expressions: _col1 (type: string)
+                      sort order: +
+                      Map-reduce partition columns: _col1 (type: string)
+                      Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col0 (type: int)
         Reducer 7 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
@@ -2020,10 +2067,13 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE
-                Reduce Output Operator
-                  sort order: 
+                Filter Operator
+                  predicate: _col0 is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE
-                  value expressions: _col0 (type: double)
+                  Reduce Output Operator
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col0 (type: double)
 
   Stage: Stage-0
     Fetch Operator
@@ -2031,7 +2081,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[44][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[47][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: select * from part where p_size >= (select min(p_size) from part_null_n0 where part_null_n0.p_type = part.p_type) AND p_retailprice <= (select max(p_retailprice) from part_null_n0)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
@@ -2388,8 +2438,9 @@ 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[36][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
-Warning: Shuffle Join MERGEJOIN[37][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 4' is a cross product
+Warning: Shuffle Join MERGEJOIN[59][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product
+Warning: Shuffle Join MERGEJOIN[58][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 8' is a cross product
+Warning: Shuffle Join MERGEJOIN[60][tables = [$hdt$_1, $hdt$_2, $hdt$_0]] in Stage 'Reducer 5' is a cross product
 PREHOOK: query: explain select key, count(*) from src where value <> (select max(value) from src) group by key having count(*) > (select count(*) from src s1 where s1.key = '90' group by s1.key )
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
@@ -2407,29 +2458,54 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 6 (XPROD_EDGE)
-        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
-        Reducer 4 <- Reducer 3 (XPROD_EDGE), Reducer 8 (XPROD_EDGE)
-        Reducer 6 <- Map 5 (CUSTOM_SIMPLE_EDGE)
-        Reducer 8 <- Map 7 (SIMPLE_EDGE)
+        Reducer 11 <- Map 10 (CUSTOM_SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE)
+        Reducer 4 <- Reducer 3 (XPROD_EDGE), Reducer 6 (XPROD_EDGE)
+        Reducer 5 <- Reducer 4 (XPROD_EDGE), Reducer 9 (XPROD_EDGE)
+        Reducer 6 <- Map 1 (SIMPLE_EDGE)
+        Reducer 8 <- Map 7 (XPROD_EDGE), Reducer 11 (XPROD_EDGE)
+        Reducer 9 <- Reducer 8 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
-                  alias: src
-                  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
-                      sort order: 
-                      Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: _col0 (type: string), _col1 (type: string)
+                  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
+                        keys: true (type: boolean)
+                        minReductionHashAggr: 0.0
+                        mode: hash
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          key expressions: _col0 (type: boolean)
+                          sort order: +
+                          Map-reduce partition columns: _col0 (type: boolean)
+                          Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                      Group By Operator
+                        aggregations: count()
+                        keys: true (type: boolean)
+                        minReductionHashAggr: 0.0
+                        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)
+                          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 10 
             Map Operator Tree:
                 TableScan
                   alias: src
@@ -2453,72 +2529,82 @@ STAGE PLANS:
         Map 7 
             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.0
-                        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)
-                          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)
+                  alias: src
+                  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
+                      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
+        Reducer 11 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: max(VALUE._col0)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: string)
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
-              Merge Join Operator
-                condition map:
-                     Inner Join 0 to 1
-                keys:
-                  0 
-                  1 
-                outputColumnNames: _col0, _col1, _col2
-                residual filter predicates: {(_col1 <> _col2)}
-                Statistics: Num rows: 500 Data size: 181000 Basic stats: COMPLETE Column stats: COMPLETE
+              Group By Operator
+                keys: KEY._col0 (type: boolean)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
-                  expressions: _col0 (type: string)
-                  outputColumnNames: _col0
-                  Statistics: Num rows: 500 Data size: 181000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
                     aggregations: count()
-                    keys: _col0 (type: string)
                     minReductionHashAggr: 0.0
                     mode: hash
-                    outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 250 Data size: 23750 Basic stats: COMPLETE Column stats: COMPLETE
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
-                      key expressions: _col0 (type: string)
-                      sort order: +
-                      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)
+                      sort order: 
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                      value expressions: _col0 (type: bigint)
         Reducer 3 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
                 aggregations: count(VALUE._col0)
-                keys: KEY._col0 (type: string)
                 mode: mergepartial
-                outputColumnNames: _col0, _col1
-                Statistics: Num rows: 250 Data size: 23750 Basic stats: COMPLETE Column stats: COMPLETE
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                Filter Operator
+                  predicate: (sq_count_check(_col0) <= 1) (type: boolean)
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    Reduce Output Operator
+                      sort order: 
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+        Reducer 4 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 
+                  1 
+                outputColumnNames: _col1
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   sort order: 
-                  Statistics: Num rows: 250 Data size: 23750 Basic stats: COMPLETE Column stats: COMPLETE
-                  value expressions: _col0 (type: string), _col1 (type: bigint)
-        Reducer 4 
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col1 (type: bigint)
+        Reducer 5 
             Execution mode: llap
             Reduce Operator Tree:
               Merge Join Operator
@@ -2527,11 +2613,11 @@ STAGE PLANS:
                 keys:
                   0 
                   1 
-                outputColumnNames: _col0, _col1, _col2
-                residual filter predicates: {(_col1 > _col2)}
+                outputColumnNames: _col1, _col2, _col3
+                residual filter predicates: {(_col3 > _col1)}
                 Statistics: Num rows: 83 Data size: 8549 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
-                  expressions: _col0 (type: string), _col1 (type: bigint)
+                  expressions: _col2 (type: string), _col3 (type: bigint)
                   outputColumnNames: _col0, _col1
                   Statistics: Num rows: 83 Data size: 7885 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
@@ -2545,18 +2631,6 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0)
-                mode: mergepartial
-                outputColumnNames: _col0
-                Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE
-                Reduce Output Operator
-                  sort order: 
-                  Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE
-                  value expressions: _col0 (type: string)
-        Reducer 8 
-            Execution mode: vectorized, llap
-            Reduce Operator Tree:
-              Group By Operator
                 aggregations: count(VALUE._col0)
                 keys: KEY._col0 (type: boolean)
                 mode: mergepartial
@@ -2564,12 +2638,64 @@ STAGE PLANS:
                 Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col1 (type: bigint)
-                  outputColumnNames: _col0
+                  outputColumnNames: _col1
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: _col1 is not null (type: boolean)
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: _col1 (type: bigint)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        sort order: 
+                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: bigint)
+        Reducer 8 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 
+                  1 
+                outputColumnNames: _col0, _col1, _col2
+                residual filter predicates: {(_col1 <> _col2)}
+                Statistics: Num rows: 500 Data size: 181000 Basic stats: COMPLETE Column stats: COMPLETE
+                Select Operator
+                  expressions: _col0 (type: string)
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 500 Data size: 181000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Group By Operator
+                    aggregations: count()
+                    keys: _col0 (type: string)
+                    minReductionHashAggr: 0.0
+                    mode: hash
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 250 Data size: 23750 Basic stats: COMPLETE Column stats: COMPLETE
+                    Reduce Output Operator
+                      key expressions: _col0 (type: string)
+                      sort order: +
+                      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)
+        Reducer 9 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                keys: KEY._col0 (type: string)
+                mode: mergepartial
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 250 Data size: 23750 Basic stats: COMPLETE Column stats: COMPLETE
+                Filter Operator
+                  predicate: _col1 is not null (type: boolean)
+                  Statistics: Num rows: 250 Data size: 23750 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     sort order: 
-                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col0 (type: bigint)
+                    Statistics: Num rows: 250 Data size: 23750 Basic stats: COMPLETE Column stats: COMPLETE
+                    value expressions: _col0 (type: string), _col1 (type: bigint)
 
   Stage: Stage-0
     Fetch Operator
@@ -2577,8 +2703,9 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[36][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
-Warning: Shuffle Join MERGEJOIN[37][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 4' is a cross product
+Warning: Shuffle Join MERGEJOIN[59][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product
+Warning: Shuffle Join MERGEJOIN[58][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 8' is a cross product
+Warning: Shuffle Join MERGEJOIN[60][tables = [$hdt$_1, $hdt$_2, $hdt$_0]] in Stage 'Reducer 5' is a cross product
 PREHOOK: query: select key, count(*) from src where value <> (select max(value) from src) group by key having count(*) > (select count(*) from src s1 where s1.key = '90' group by s1.key )
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
@@ -2597,7 +2724,7 @@ POSTHOOK: Input: default@src
 468	4
 469	5
 489	4
-Warning: Shuffle Join MERGEJOIN[22][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[25][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: explain select sum(p_retailprice) from part group by p_type having sum(p_retailprice) > (select max(pp.p_retailprice) from part pp)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
@@ -2678,10 +2805,13 @@ STAGE PLANS:
                   expressions: _col1 (type: double)
                   outputColumnNames: _col1
                   Statistics: Num rows: 13 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    sort order: 
+                  Filter Operator
+                    predicate: _col1 is not null (type: boolean)
                     Statistics: Num rows: 13 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col1 (type: double)
+                    Reduce Output Operator
+                      sort order: 
+                      Statistics: Num rows: 13 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
+                      value expressions: _col1 (type: double)
         Reducer 3 
             Execution mode: llap
             Reduce Operator Tree:
@@ -2713,10 +2843,13 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                Reduce Output Operator
-                  sort order: 
+                Filter Operator
+                  predicate: _col0 is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                  value expressions: _col0 (type: double)
+                  Reduce Output Operator
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    value expressions: _col0 (type: double)
 
   Stage: Stage-0
     Fetch Operator
@@ -2724,7 +2857,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[22][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[25][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: select sum(p_retailprice) from part group by p_type having sum(p_retailprice) > (select max(pp.p_retailprice) from part pp)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
@@ -2735,7 +2868,7 @@ POSTHOOK: Input: default@part
 #### A masked pattern was here ####
 2346.3
 3461.37
-Warning: Shuffle Join MERGEJOIN[43][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+Warning: Shuffle Join MERGEJOIN[48][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
 PREHOOK: query: explain select * from part where p_size > (select count(p_name) from part INTERSECT select count(p_brand) from part)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
@@ -2765,15 +2898,19 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: part
+                  filterExpr: UDFToLong(p_size) is not null (type: boolean)
                   Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), UDFToLong(p_size) (type: bigint)
-                    outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
-                    Statistics: Num rows: 26 Data size: 16302 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      sort order: 
+                  Filter Operator
+                    predicate: UDFToLong(p_size) is not null (type: boolean)
+                    Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), UDFToLong(p_size) (type: bigint)
+                      outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
                       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), _col9 (type: bigint)
+                      Reduce Output Operator
+                        sort order: 
+                        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), _col9 (type: bigint)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 3 
@@ -2871,19 +3008,22 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                Group By Operator
-                  aggregations: count()
-                  keys: _col0 (type: bigint)
-                  minReductionHashAggr: 0.0
-                  mode: hash
-                  outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: bigint)
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: bigint)
+                Filter Operator
+                  predicate: _col0 is not null (type: boolean)
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                  Group By Operator
+                    aggregations: count()
+                    keys: _col0 (type: bigint)
+                    minReductionHashAggr: 0.0
+                    mode: hash
+                    outputColumnNames: _col0, _col1
                     Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col1 (type: bigint)
+                    Reduce Output Operator
+                      key expressions: _col0 (type: bigint)
+                      sort order: +
+                      Map-reduce partition columns: _col0 (type: bigint)
+                      Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+                      value expressions: _col1 (type: bigint)
         Reducer 5 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
@@ -2934,19 +3074,22 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                Group By Operator
-                  aggregations: count()
-                  keys: _col0 (type: bigint)
-                  minReductionHashAggr: 0.0
-                  mode: hash
-                  outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: bigint)
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: bigint)
+                Filter Operator
+                  predicate: _col0 is not null (type: boolean)
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                  Group By Operator
+                    aggregations: count()
+                    keys: _col0 (type: bigint)
+                    minReductionHashAggr: 0.0
+                    mode: hash
+                    outputColumnNames: _col0, _col1
                     Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col1 (type: bigint)
+                    Reduce Output Operator
+                      key expressions: _col0 (type: bigint)
+                      sort order: +
+                      Map-reduce partition columns: _col0 (type: bigint)
+                      Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+                      value expressions: _col1 (type: bigint)
         Union 6 
             Vertex: Union 6
 
@@ -2956,7 +3099,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[43][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+Warning: Shuffle Join MERGEJOIN[48][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
 PREHOOK: query: select * from part where p_size > (select count(p_name) from part INTERSECT select count(p_brand) from part)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
@@ -4111,10 +4254,10 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: lineitem
-                  filterExpr: l_partkey is not null (type: boolean)
+                  filterExpr: (l_partkey is not null and l_quantity 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 (type: boolean)
+                    predicate: (l_partkey is not null and l_quantity is not null) (type: boolean)
                     Statistics: Num rows: 100 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: l_partkey (type: int), l_quantity (type: double), l_extendedprice (type: double)
@@ -4239,16 +4382,19 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 50 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: _col0 (type: int), (_col1 / _col2) (type: double)
-                  outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 50 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: int)
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: int)
+                Filter Operator
+                  predicate: (_col1 is not null and _col2 is not null) (type: boolean)
+                  Statistics: Num rows: 50 Data size: 1000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: _col0 (type: int), (_col1 / _col2) (type: double)
+                    outputColumnNames: _col0, _col1
                     Statistics: Num rows: 50 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col1 (type: double)
+                    Reduce Output Operator
+                      key expressions: _col0 (type: int)
+                      sort order: +
+                      Map-reduce partition columns: _col0 (type: int)
+                      Statistics: Num rows: 50 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE
+                      value expressions: _col1 (type: double)
 
   Stage: Stage-0
     Fetch Operator
@@ -4756,10 +4902,10 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: emps_n4
-                  filterExpr: deptno is not null (type: boolean)
+                  filterExpr: (deptno is not null and name is not null) (type: boolean)
                   Statistics: Num rows: 5 Data size: 1650 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: deptno is not null (type: boolean)
+                    predicate: (deptno is not null and name is not null) (type: boolean)
                     Statistics: Num rows: 5 Data size: 1650 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: empno (type: int), name (type: string), deptno (type: int), gender (type: string), city (type: string), empid (type: int), age (type: int), slacker (type: boolean), manager (type: boolean), joinedat (type: date)
@@ -4829,16 +4975,19 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: _col1 (type: string), _col0 (type: int)
-                  outputColumnNames: _col0, _col1
+                Filter Operator
+                  predicate: _col1 is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    key expressions: _col1 (type: int)
-                    sort order: +
-                    Map-reduce partition columns: _col1 (type: int)
+                  Select Operator
+                    expressions: _col1 (type: string), _col0 (type: int)
+                    outputColumnNames: _col0, _col1
                     Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col0 (type: string)
+                    Reduce Output Operator
+                      key expressions: _col1 (type: int)
+                      sort order: +
+                      Map-reduce partition columns: _col1 (type: int)
+                      Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: COMPLETE
+                      value expressions: _col0 (type: string)
 
   Stage: Stage-0
     Fetch Operator
@@ -5048,7 +5197,7 @@ POSTHOOK: Input: default@emps_n4
 110	John	40	M	Vancouver	2	NULL	false	true	2002-05-03
 120	Wilma	20	F	NULL	1	5	NULL	true	2005-09-07
 130	Alice	40	F	Vancouver	2	NULL	false	true	2007-01-01
-Warning: Shuffle Join MERGEJOIN[39][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[41][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: explain select * from emps_n4 where deptno <> (select sum(deptno) from depts_n3 where depts_n3.name = emps_n4.name) and empno > (select count(name) from depts_n3)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@depts_n3
@@ -5078,10 +5227,10 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: emps_n4
-                  filterExpr: name is not null (type: boolean)
+                  filterExpr: (name is not null and UDFToLong(empno) is not null) (type: boolean)
                   Statistics: Num rows: 5 Data size: 1650 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: name is not null (type: boolean)
+                    predicate: (UDFToLong(empno) is not null and name is not null) (type: boolean)
                     Statistics: Num rows: 5 Data size: 1650 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: empno (type: int), name (type: string), deptno (type: int), gender (type: string), city (type: string), empid (type: int), age (type: int), slacker (type: boolean), manager (type: boolean), joinedat (type: date), UDFToLong(deptno) (type: bigint)
@@ -5206,10 +5355,13 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                Reduce Output Operator
-                  sort order: 
+                Filter Operator
+                  predicate: _col0 is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                  value expressions: _col0 (type: bigint)
+                  Reduce Output Operator
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    value expressions: _col0 (type: bigint)
 
   Stage: Stage-0
     Fetch Operator
@@ -5217,7 +5369,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[38][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[41][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: select * from emps_n4 where deptno <> (select count(deptno) from depts_n3 where depts_n3.name = emps_n4.name) and empno > (select count(name) from depts_n3)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@depts_n3
@@ -5249,7 +5401,7 @@ POSTHOOK: query: drop table EMPS_n4
 POSTHOOK: type: DROPTABLE
 POSTHOOK: Input: default@emps_n4
 POSTHOOK: Output: default@emps_n4
-Warning: Shuffle Join MERGEJOIN[23][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[26][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: explain
  select key, count(*)
 from src
@@ -5328,10 +5480,13 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 250 Data size: 23750 Basic stats: COMPLETE Column stats: COMPLETE
-                Reduce Output Operator
-                  sort order: 
+                Filter Operator
+                  predicate: _col1 is not null (type: boolean)
                   Statistics: Num rows: 250 Data size: 23750 Basic stats: COMPLETE Column stats: COMPLETE
-                  value expressions: _col0 (type: string), _col1 (type: bigint)
+                  Reduce Output Operator
+                    sort order: 
+                    Statistics: Num rows: 250 Data size: 23750 Basic stats: COMPLETE Column stats: COMPLETE
+                    value expressions: _col0 (type: string), _col1 (type: bigint)
         Reducer 3 
             Execution mode: llap
             Reduce Operator Tree:
@@ -5363,10 +5518,13 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                Reduce Output Operator
-                  sort order: 
+                Filter Operator
+                  predicate: _col0 is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                  value expressions: _col0 (type: bigint)
+                  Reduce Output Operator
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    value expressions: _col0 (type: bigint)
 
   Stage: Stage-0
     Fetch Operator
@@ -5374,7 +5532,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[23][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[26][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: select key, count(*)
 from src
 group by key
@@ -5399,7 +5557,7 @@ POSTHOOK: Input: default@src
 468	4
 469	5
 489	4
-Warning: Shuffle Join MERGEJOIN[51][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 5' is a cross product
+Warning: Shuffle Join MERGEJOIN[54][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 5' is a cross product
 PREHOOK: query: explain
 select key, value, count(*)
 from src b
@@ -5538,10 +5696,13 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 41 Data size: 7626 Basic stats: COMPLETE Column stats: COMPLETE
-                Reduce Output Operator
-                  sort order: 
+                Filter Operator
+                  predicate: _col2 is not null (type: boolean)
                   Statistics: Num rows: 41 Data size: 7626 Basic stats: COMPLETE Column stats: COMPLETE
-                  value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: bigint)
+                  Reduce Output Operator
+                    sort order: 
+                    Statistics: Num rows: 41 Data size: 7626 Basic stats: COMPLETE Column stats: COMPLETE
+                    value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: bigint)
         Reducer 5 
             Execution mode: llap
             Reduce Operator Tree:
@@ -5573,10 +5734,13 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                Reduce Output Operator
-                  sort order: 
+                Filter Operator
+                  predicate: _col0 is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                  value expressions: _col0 (type: bigint)
+                  Reduce Output Operator
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    value expressions: _col0 (type: bigint)
 
   Stage: Stage-0
     Fetch Operator
@@ -5584,7 +5748,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[51][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 5' is a cross product
+Warning: Shuffle Join MERGEJOIN[54][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 5' is a cross product
 PREHOOK: query: select key, value, count(*)
 from src b
 where b.key in (select key from src where src.key > '8')
@@ -5601,8 +5765,8 @@ having count(*) > (select count(*) from src s1 where s1.key > '9' )
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@src
 #### A masked pattern was here ####
-Warning: Shuffle Join MERGEJOIN[35][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
-Warning: Shuffle Join MERGEJOIN[36][tables = [$hdt$_1, $hdt$_2, $hdt$_0]] in Stage 'Reducer 4' is a cross product
+Warning: Shuffle Join MERGEJOIN[38][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[39][tables = [$hdt$_1, $hdt$_2, $hdt$_0]] in Stage 'Reducer 4' is a cross product
 PREHOOK: query: explain  select * from part where p_size > (select max(p_size) from part group by p_type)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
@@ -5677,15 +5841,19 @@ STAGE PLANS:
             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
-                  Select Operator
-                    expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string)
-                    outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
+                  Filter Operator
+                    predicate: p_size is not null (type: boolean)
                     Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      sort order: 
+                    Select Operator
+                      expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string)
+                      outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                       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), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
+                      Reduce Output Operator
+                        sort order: 
+                        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), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
@@ -5699,12 +5867,19 @@ STAGE PLANS:
                 Statistics: Num rows: 13 Data size: 1404 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col1 (type: int)
-                  outputColumnNames: _col0
+                  outputColumnNames: _col1
                   Statistics: Num rows: 13 Data size: 52 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    sort order: 
+                  Filter Operator
+                    predicate: _col1 is not null (type: boolean)
                     Statistics: Num rows: 13 Data size: 52 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col0 (type: int)
+                    Select Operator
+                      expressions: _col1 (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 13 Data size: 52 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        sort order: 
+                        Statistics: Num rows: 13 Data size: 52 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: int)
         Reducer 3 
             Execution mode: llap
             Reduce Operator Tree:
@@ -5813,10 +5988,10 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: part
-                  filterExpr: p_type is not null (type: boolean)
+                  filterExpr: (p_type 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_type is not null (type: boolean)
+                    predicate: (p_size is not null and p_type is not null) (type: boolean)
                     Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string)
@@ -5952,15 +6127,22 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 13 Data size: 1404 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
-                  expressions: _col1 (type: int), _col0 (type: string)
-                  outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 13 Data size: 1404 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    key expressions: _col1 (type: string)
-                    sort order: +
-                    Map-reduce partition columns: _col1 (type: string)
-                    Statistics: Num rows: 13 Data size: 1404 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col0 (type: int)
+                  expressions: _col0 (type: string), _col1 (type: int)
+                  outputColumnNames: _col1, _col2
+                  Statistics: Num rows: 13 Data size: 2756 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: _col2 is not null (type: boolean)
+                    Statistics: Num rows: 13 Data size: 2756 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: _col2 (type: int), _col1 (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 13 Data size: 1404 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col1 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col1 (type: string)
+                        Statistics: Num rows: 13 Data size: 1404 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: int)
 
   Stage: Stage-0
     Fetch Operator
@@ -6819,7 +7001,8 @@ POSTHOOK: query: drop table tempty_n0
 POSTHOOK: type: DROPTABLE
 POSTHOOK: Input: default@tempty_n0
 POSTHOOK: Output: default@tempty_n0
-Warning: Shuffle Join MERGEJOIN[23][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[43][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product
+Warning: Shuffle Join MERGEJOIN[44][tables = [$hdt$_1, $hdt$_2, $hdt$_0]] in Stage 'Reducer 5' is a cross product
 PREHOOK: query: explain select key, count(*) from src group by key having count(*) >
     (select count(*) from src s1 group by 4)
 PREHOOK: type: QUERY
@@ -6840,53 +7023,67 @@ STAGE PLANS:
 #### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (XPROD_EDGE), Reducer 5 (XPROD_EDGE)
-        Reducer 5 <- Map 4 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE)
+        Reducer 4 <- Reducer 3 (XPROD_EDGE), Reducer 6 (XPROD_EDGE)
+        Reducer 5 <- Reducer 4 (XPROD_EDGE), Reducer 8 (XPROD_EDGE)
+        Reducer 6 <- Map 1 (SIMPLE_EDGE)
+        Reducer 8 <- Map 7 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
-                  alias: src
-                  Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
+                  alias: s1
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
-                    expressions: key (type: string)
-                    outputColumnNames: key
-                    Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                    Group By Operator
+                      keys: true (type: boolean)
+                      minReductionHashAggr: 0.0
+                      mode: hash
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: boolean)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: boolean)
+                        Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
                       aggregations: count()
-                      keys: key (type: string)
+                      keys: true (type: boolean)
                       minReductionHashAggr: 0.0
                       mode: hash
                       outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 250 Data size: 23750 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
-                        key expressions: _col0 (type: string)
+                        key expressions: _col0 (type: boolean)
                         sort order: +
-                        Map-reduce partition columns: _col0 (type: string)
-                        Statistics: Num rows: 250 Data size: 23750 Basic stats: COMPLETE Column stats: COMPLETE
+                        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 4 
+        Map 7 
             Map Operator Tree:
                 TableScan
-                  alias: s1
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  alias: src
+                  Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
-                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                    expressions: key (type: string)
+                    outputColumnNames: key
+                    Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
                       aggregations: count()
-                      keys: true (type: boolean)
+                      keys: key (type: string)
                       minReductionHashAggr: 0.0
                       mode: hash
                       outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 250 Data size: 23750 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
-                        key expressions: _col0 (type: boolean)
+                        key expressions: _col0 (type: string)
                         sort order: +
-                        Map-reduce partition columns: _col0 (type: boolean)
-                        Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                        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
@@ -6894,16 +7091,54 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
+                keys: KEY._col0 (type: boolean)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                Select Operator
+                  Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                  Group By Operator
+                    aggregations: count()
+                    minReductionHashAggr: 0.0
+                    mode: hash
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    Reduce Output Operator
+                      sort order: 
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                      value expressions: _col0 (type: bigint)
+        Reducer 3 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
                 aggregations: count(VALUE._col0)
-                keys: KEY._col0 (type: string)
                 mode: mergepartial
-                outputColumnNames: _col0, _col1
-                Statistics: Num rows: 250 Data size: 23750 Basic stats: COMPLETE Column stats: COMPLETE
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                Filter Operator
+                  predicate: (sq_count_check(_col0) <= 1) (type: boolean)
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    Reduce Output Operator
+                      sort order: 
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+        Reducer 4 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 
+                  1 
+                outputColumnNames: _col1
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   sort order: 
-                  Statistics: Num rows: 250 Data size: 23750 Basic stats: COMPLETE Column stats: COMPLETE
-                  value expressions: _col0 (type: string), _col1 (type: bigint)
-        Reducer 3 
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col1 (type: bigint)
+        Reducer 5 
             Execution mode: llap
             Reduce Operator Tree:
               Merge Join Operator
@@ -6912,11 +7147,11 @@ STAGE PLANS:
                 keys:
                   0 
                   1 
-                outputColumnNames: _col0, _col1, _col2
-                residual filter predicates: {(_col1 > _col2)}
+                outputColumnNames: _col1, _col2, _col3
+                residual filter predicates: {(_col3 > _col1)}
                 Statistics: Num rows: 83 Data size: 8549 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
-                  expressions: _col0 (type: string), _col1 (type: bigint)
+                  expressions: _col2 (type: string), _col3 (type: bigint)
                   outputColumnNames: _col0, _col1
                   Statistics: Num rows: 83 Data size: 7885 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
@@ -6926,7 +7161,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 6 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -6937,12 +7172,35 @@ STAGE PLANS:
                 Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col1 (type: bigint)
-                  outputColumnNames: _col0
+                  outputColumnNames: _col1
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: _col1 is not null (type: boolean)
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: _col1 (type: bigint)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        sort order: 
+                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: bigint)
+        Reducer 8 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                keys: KEY._col0 (type: string)
+                mode: mergepartial
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 250 Data size: 23750 Basic stats: COMPLETE Column stats: COMPLETE
+                Filter Operator
+                  predicate: _col1 is not null (type: boolean)
+                  Statistics: Num rows: 250 Data size: 23750 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     sort order: 
-                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col0 (type: bigint)
+                    Statistics: Num rows: 250 Data size: 23750 Basic stats: COMPLETE Column stats: COMPLETE
+                    value expressions: _col0 (type: string), _col1 (type: bigint)
 
   Stage: Stage-0
     Fetch Operator
@@ -6950,7 +7208,8 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[25][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[47][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 6' is a cross product
+Warning: Shuffle Join MERGEJOIN[48][tables = [$hdt$_1, $hdt$_2, $hdt$_0]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: explain select key, count(*) from src group by key having count(*) >
     (select count(*) from src s1 where s1.key = '90' group by s1.key )
 PREHOOK: type: QUERY
@@ -6971,8 +7230,11 @@ STAGE PLANS:
 #### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (XPROD_EDGE), Reducer 4 (XPROD_EDGE)
+        Reducer 3 <- Reducer 2 (XPROD_EDGE), Reducer 6 (XPROD_EDGE)
         Reducer 4 <- Map 1 (SIMPLE_EDGE)
+        Reducer 5 <- Reducer 4 (CUSTOM_SIMPLE_EDGE)
+        Reducer 6 <- Reducer 5 (XPROD_EDGE), Reducer 7 (XPROD_EDGE)
+        Reducer 7 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -7003,6 +7265,22 @@ STAGE PLANS:
                     Select Operator
                       Statistics: Num rows: 2 Data size: 174 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
+                        keys: true (type: boolean)
+                        minReductionHashAggr: 0.0
+                        mode: hash
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          key expressions: _col0 (type: boolean)
+                          sort order: +
+                          Map-reduce partition columns: _col0 (type: boolean)
+                          Statistics: Num rows: 1 Data size: 4 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.0
@@ -7026,10 +7304,13 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 250 Data size: 23750 Basic stats: COMPLETE Column stats: COMPLETE
-                Reduce Output Operator
-                  sort order: 
+                Filter Operator
+                  predicate: _col1 is not null (type: boolean)
                   Statistics: Num rows: 250 Data size: 23750 Basic stats: COMPLETE Column stats: COMPLETE
-                  value expressions: _col0 (type: string), _col1 (type: bigint)
+                  Reduce Output Operator
+                    sort order: 
+                    Statistics: Num rows: 250 Data size: 23750 Basic stats: COMPLETE Column stats: COMPLETE
+                    value expressions: _col0 (type: string), _col1 (type: bigint)
         Reducer 3 
             Execution mode: llap
             Reduce Operator Tree:
@@ -7039,11 +7320,11 @@ STAGE PLANS:
                 keys:
                   0 
                   1 
-                outputColumnNames: _col0, _col1, _col2
-                residual filter predicates: {(_col1 > _col2)}
+                outputColumnNames: _col1, _col2, _col3
+                residual filter predicates: {(_col3 > _col1)}
                 Statistics: Num rows: 83 Data size: 8549 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
-                  expressions: _col0 (type: string), _col1 (type: bigint)
+                  expressions: _col2 (type: string), _col3 (type: bigint)
                   outputColumnNames: _col0, _col1
                   Statistics: Num rows: 83 Data size: 7885 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
@@ -7057,6 +7338,57 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
+                keys: KEY._col0 (type: boolean)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                Select Operator
+                  Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                  Group By Operator
+                    aggregations: count()
+                    minReductionHashAggr: 0.0
+                    mode: hash
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    Reduce Output Operator
+                      sort order: 
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                      value expressions: _col0 (type: bigint)
+        Reducer 5 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                Filter Operator
+                  predicate: (sq_count_check(_col0) <= 1) (type: boolean)
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    Reduce Output Operator
+                      sort order: 
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+        Reducer 6 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 
+                  1 
+                outputColumnNames: _col1
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col1 (type: bigint)
+        Reducer 7 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
                 aggregations: count(VALUE._col0)
                 keys: KEY._col0 (type: boolean)
                 mode: mergepartial
@@ -7064,12 +7396,19 @@ STAGE PLANS:
                 Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col1 (type: bigint)
-                  outputColumnNames: _col0
+                  outputColumnNames: _col1
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    sort order: 
+                  Filter Operator
+                    predicate: _col1 is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col0 (type: bigint)
+                    Select Operator
+                      expressions: _col1 (type: bigint)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        sort order: 
+                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: bigint)
 
   Stage: Stage-0
     Fetch Operator
@@ -7103,7 +7442,7 @@ POSTHOOK: query: CREATE TABLE `date_dim`(
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@date_dim
-Warning: Shuffle Join MERGEJOIN[42][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+Warning: Shuffle Join MERGEJOIN[45][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
 PREHOOK: query: explain cbo with avg_sales as
  (select avg(quantity*list_price) average_sales
   from (select ss_quantity quantity
@@ -7134,20 +7473,23 @@ CBO PLAN:
 HiveProject(ss_sold_date_sk=[$0], ss_quantity=[$1], ss_list_price=[$2])
   HiveJoin(condition=[>($2, $3)], joinType=[inner], algorithm=[none], cost=[not available])
     HiveProject(ss_sold_date_sk=[$0], ss_quantity=[$1], ss_list_price=[$2])
-      HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+      HiveFilter(condition=[IS NOT NULL($2)])
+        HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
     HiveProject($f0=[/($0, $1)])
-      HiveAggregate(group=[{}], agg#0=[sum($0)], agg#1=[count($0)])
-        HiveProject($f0=[*(CAST($1):DECIMAL(10, 0), $2)])
-          HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-            HiveProject(ss_sold_date_sk=[$0], ss_quantity=[$1], ss_list_price=[$2])
-              HiveFilter(condition=[IS NOT NULL($0)])
-                HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-            HiveProject(d_date_sk=[$0])
-              HiveFilter(condition=[AND(BETWEEN(false, $1, 1999, 2001), IS NOT NULL($0))])
-                HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+      HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($1))])
+        HiveProject($f0=[$0], $f1=[$1])
+          HiveAggregate(group=[{}], agg#0=[sum($0)], agg#1=[count($0)])
+            HiveProject($f0=[*(CAST($1):DECIMAL(10, 0), $2)])
+              HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                HiveProject(ss_sold_date_sk=[$0], ss_quantity=[$1], ss_list_price=[$2])
+                  HiveFilter(condition=[IS NOT NULL($0)])
+                    HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                HiveProject(d_date_sk=[$0])
+                  HiveFilter(condition=[AND(BETWEEN(false, $1, 1999, 2001), IS NOT NULL($0))])
+                    HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
 
-Warning: Shuffle Join MERGEJOIN[74][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
-Warning: Shuffle Join MERGEJOIN[76][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[77][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+Warning: Shuffle Join MERGEJOIN[79][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: explain cbo with avg_sales as
  (select avg(quantity*list_price) over( partition by list_price) average_sales
   from (select ss_quantity quantity
@@ -7179,15 +7521,18 @@ HiveProject(ss_sold_date_sk=[$0], ss_quantity=[$1], ss_list_price=[$2])
   HiveJoin(condition=[true], joinType=[inner], algorithm=[none], cost=[not available])
     HiveJoin(condition=[>($2, $3)], joinType=[inner], algorithm=[none], cost=[not available])
       HiveProject(ss_sold_date_sk=[$0], ss_quantity=[$1], ss_list_price=[$2])
-        HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-      HiveProject(avg_window_0=[avg(*(CAST($1):DECIMAL(10, 0), $2)) OVER (PARTITION BY $2 ORDER BY $2 NULLS FIRST ROWS BETWEEN 2147483647 FOLLOWING AND 2147483647 PRECEDING)])
-        HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-          HiveProject(ss_sold_date_sk=[$0], ss_quantity=[$1], ss_list_price=[$2])
-            HiveFilter(condition=[IS NOT NULL($0)])
-              HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-          HiveProject(d_date_sk=[$0])
-            HiveFilter(condition=[AND(BETWEEN(false, $1, 1999, 2001), IS NOT NULL($0))])
-              HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+        HiveFilter(condition=[IS NOT NULL($2)])
+          HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+      HiveProject(avg_window_0=[$0])
+        HiveFilter(condition=[IS NOT NULL($0)])
+          HiveProject(avg_window_0=[avg(*(CAST($1):DECIMAL(10, 0), $2)) OVER (PARTITION BY $2 ORDER BY $2 NULLS FIRST ROWS BETWEEN 2147483647 FOLLOWING AND 2147483647 PRECEDING)])
+            HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+              HiveProject(ss_sold_date_sk=[$0], ss_quantity=[$1], ss_list_price=[$2])
+                HiveFilter(condition=[IS NOT NULL($0)])
+                  HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+              HiveProject(d_date_sk=[$0])
+                HiveFilter(condition=[AND(BETWEEN(false, $1, 1999, 2001), IS NOT NULL($0))])
+                  HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
     HiveProject(cnt=[$0])
       HiveFilter(condition=[<=(sq_count_check($0), 1)])
         HiveProject(cnt=[$0])
diff --git a/ql/src/test/results/clientpositive/llap/subquery_select.q.out b/ql/src/test/results/clientpositive/llap/subquery_select.q.out
index 75e3428..fc70407 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_select.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_select.q.out
@@ -4227,7 +4227,7 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@part
 #### A masked pattern was here ####
 true
-Warning: Shuffle Join MERGEJOIN[53][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product
+Warning: Shuffle Join MERGEJOIN[56][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product
 PREHOOK: query: explain select o.p_size, (select count(distinct p_type) from part p where p.p_partkey = o.p_partkey) tmp
     FROM part o right join (select * from part where p_size > (select avg(p_size) from part)) t on t.p_partkey = o.p_partkey
 PREHOOK: type: QUERY
@@ -4258,6 +4258,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: o
+                  filterExpr: (p_partkey is not null or UDFToDouble(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 (type: boolean)
@@ -4272,14 +4273,17 @@ STAGE PLANS:
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 26 Data size: 208 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: int)
-                  Select Operator
-                    expressions: p_partkey (type: int), UDFToDouble(p_size) (type: double)
-                    outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 26 Data size: 312 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      sort order: 
+                  Filter Operator
+                    predicate: UDFToDouble(p_size) is not null (type: boolean)
+                    Statistics: Num rows: 26 Data size: 208 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: p_partkey (type: int), UDFToDouble(p_size) (type: double)
+                      outputColumnNames: _col0, _col1
                       Statistics: Num rows: 26 Data size: 312 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: _col0 (type: int), _col1 (type: double)
+                      Reduce Output Operator
+                        sort order: 
+                        Statistics: Num rows: 26 Data size: 312 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: int), _col1 (type: double)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 5 
@@ -4393,14 +4397,17 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: (UDFToDouble(_col0) / _col1) (type: double)
-                  outputColumnNames: _col0
-                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    sort order: 
+                Filter Operator
+                  predicate: (UDFToDouble(_col0) is not null and _col1 is not null) (type: boolean)
+                  Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: (UDFToDouble(_col0) / _col1) (type: double)
+                    outputColumnNames: _col0
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col0 (type: double)
+                    Reduce Output Operator
+                      sort order: 
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                      value expressions: _col0 (type: double)
         Reducer 8 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
@@ -4432,7 +4439,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[56][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product
 PREHOOK: query: select o.p_size, (select count(distinct p_type) from part p where p.p_partkey = o.p_partkey) tmp
     FROM part o right join (select * from part where p_size > (select avg(p_size) from part)) t on t.p_partkey = o.p_partkey
 PREHOOK: type: QUERY
diff --git a/ql/src/test/results/clientpositive/llap/transitive_not_null.q.out b/ql/src/test/results/clientpositive/llap/transitive_not_null.q.out
index a0089ac..417bb45 100644
--- a/ql/src/test/results/clientpositive/llap/transitive_not_null.q.out
+++ b/ql/src/test/results/clientpositive/llap/transitive_not_null.q.out
@@ -215,7 +215,7 @@ HiveJoin(condition=[AND(=($0, $2), >($1, $3))], joinType=[left], algorithm=[none
   HiveProject(i1=[$0], j1=[$1])
     HiveTableScan(table=[[default, tbl_1]], table:alias=[tbl_1])
   HiveProject(i2=[$0], j2=[$1])
-    HiveFilter(condition=[IS NOT NULL($0)])
+    HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($1))])
       HiveTableScan(table=[[default, tbl_2]], table:alias=[tbl_2])
 
 PREHOOK: query: select * from tbl_1 left join tbl_2 on tbl_1.i1 = tbl_2.i2 AND tbl_1.j1>tbl_2.j2
@@ -245,7 +245,7 @@ POSTHOOK: Input: default@tbl_2
 CBO PLAN:
 HiveJoin(condition=[AND(=($0, $2), >($1, $3))], joinType=[right], algorithm=[none], cost=[not available])
   HiveProject(i1=[$0], j1=[$1])
-    HiveFilter(condition=[IS NOT NULL($0)])
+    HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($1))])
       HiveTableScan(table=[[default, tbl_1]], table:alias=[tbl_1])
   HiveProject(i2=[$0], j2=[$1])
     HiveTableScan(table=[[default, tbl_2]], table:alias=[tbl_2])
diff --git a/ql/src/test/results/clientpositive/perf/spark/query1.q.out b/ql/src/test/results/clientpositive/perf/spark/query1.q.out
index b2ebbf6..7da93fa 100644
--- a/ql/src/test/results/clientpositive/perf/spark/query1.q.out
+++ b/ql/src/test/results/clientpositive/perf/spark/query1.q.out
@@ -66,7 +66,7 @@ STAGE PLANS:
     Spark
 #### A masked pattern was here ####
       Vertices:
-        Map 8 
+        Map 1 
             Map Operator Tree:
                 TableScan
                   alias: store
@@ -81,8 +81,8 @@ STAGE PLANS:
                       Statistics: Num rows: 852 Data size: 1628138 Basic stats: COMPLETE Column stats: NONE
                       Spark HashTable Sink Operator
                         keys:
-                          0 _col1 (type: int)
-                          1 _col0 (type: int)
+                          0 _col0 (type: int)
+                          1 _col1 (type: int)
             Execution mode: vectorized
             Local Work:
               Map Reduce Local Work
@@ -92,33 +92,13 @@ STAGE PLANS:
       Edges:
         Reducer 11 <- Map 10 (PARTITION-LEVEL SORT, 36), Map 13 (PARTITION-LEVEL SORT, 36)
         Reducer 12 <- Reducer 11 (GROUP PARTITION-LEVEL SORT, 39)
-        Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 36), Map 7 (PARTITION-LEVEL SORT, 36)
-        Reducer 3 <- Reducer 2 (GROUP, 39)
-        Reducer 4 <- Map 9 (PARTITION-LEVEL SORT, 559), Reducer 3 (PARTITION-LEVEL SORT, 559)
-        Reducer 5 <- Reducer 12 (PARTITION-LEVEL SORT, 601), Reducer 4 (PARTITION-LEVEL SORT, 601)
-        Reducer 6 <- Reducer 5 (SORT, 1)
+        Reducer 3 <- Map 2 (PARTITION-LEVEL SORT, 36), Map 8 (PARTITION-LEVEL SORT, 36)
+        Reducer 4 <- Reducer 3 (GROUP, 39)
+        Reducer 5 <- Map 9 (PARTITION-LEVEL SORT, 559), Reducer 4 (PARTITION-LEVEL SORT, 559)
+        Reducer 6 <- Reducer 12 (PARTITION-LEVEL SORT, 601), Reducer 5 (PARTITION-LEVEL SORT, 601)
+        Reducer 7 <- Reducer 6 (SORT, 1)
 #### A masked pattern was here ####
       Vertices:
-        Map 1 
-            Map Operator Tree:
-                TableScan
-                  alias: store_returns
-                  filterExpr: (sr_returned_date_sk is not null and sr_store_sk is not null and sr_customer_sk is not null) (type: boolean)
-                  Statistics: Num rows: 57591150 Data size: 4462194832 Basic stats: COMPLETE Column stats: NONE
-                  Filter Operator
-                    predicate: (sr_customer_sk is not null and sr_returned_date_sk is not null and sr_store_sk is not null) (type: boolean)
-                    Statistics: Num rows: 57591150 Data size: 4462194832 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: sr_returned_date_sk (type: int), sr_customer_sk (type: int), sr_store_sk (type: int), sr_fee (type: decimal(7,2))
-                      outputColumnNames: _col0, _col1, _col2, _col3
-                      Statistics: Num rows: 57591150 Data size: 4462194832 Basic stats: COMPLETE Column stats: NONE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: int)
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 57591150 Data size: 4462194832 Basic stats: COMPLETE Column stats: NONE
-                        value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: decimal(7,2))
-            Execution mode: vectorized
         Map 10 
             Map Operator Tree:
                 TableScan
@@ -158,7 +138,27 @@ STAGE PLANS:
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 36524 Data size: 40870356 Basic stats: COMPLETE Column stats: NONE
             Execution mode: vectorized
-        Map 7 
+        Map 2 
+            Map Operator Tree:
+                TableScan
+                  alias: store_returns
+                  filterExpr: (sr_returned_date_sk is not null and sr_store_sk is not null and sr_customer_sk is not null) (type: boolean)
+                  Statistics: Num rows: 57591150 Data size: 4462194832 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: (sr_customer_sk is not null and sr_returned_date_sk is not null and sr_store_sk is not null) (type: boolean)
+                    Statistics: Num rows: 57591150 Data size: 4462194832 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: sr_returned_date_sk (type: int), sr_customer_sk (type: int), sr_store_sk (type: int), sr_fee (type: decimal(7,2))
+                      outputColumnNames: _col0, _col1, _col2, _col3
+                      Statistics: Num rows: 57591150 Data size: 4462194832 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 57591150 Data size: 4462194832 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: decimal(7,2))
+            Execution mode: vectorized
+        Map 8 
             Map Operator Tree:
                 TableScan
                   alias: date_dim
@@ -239,17 +239,20 @@ STAGE PLANS:
                     mode: complete
                     outputColumnNames: _col0, _col1, _col2
                     Statistics: Num rows: 15837566 Data size: 1227103566 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: ((_col1 / _col2) * 1.2) (type: decimal(38,11)), _col0 (type: int)
-                      outputColumnNames: _col0, _col1
+                    Filter Operator
+                      predicate: (_col1 is not null and _col2 is not null) (type: boolean)
                       Statistics: Num rows: 15837566 Data size: 1227103566 Basic stats: COMPLETE Column stats: NONE
-                      Reduce Output Operator
-                        key expressions: _col1 (type: int)
-                        sort order: +
-                        Map-reduce partition columns: _col1 (type: int)
+                      Select Operator
+                        expressions: ((_col1 / _col2) * 1.2) (type: decimal(38,11)), _col0 (type: int)
+                        outputColumnNames: _col0, _col1
                         Statistics: Num rows: 15837566 Data size: 1227103566 Basic stats: COMPLETE Column stats: NONE
-                        value expressions: _col0 (type: decimal(38,11))
-        Reducer 2 
+                        Reduce Output Operator
+                          key expressions: _col1 (type: int)
+                          sort order: +
+                          Map-reduce partition columns: _col1 (type: int)
+                          Statistics: Num rows: 15837566 Data size: 1227103566 Basic stats: COMPLETE Column stats: NONE
+                          value expressions: _col0 (type: decimal(38,11))
+        Reducer 3 
             Reduce Operator Tree:
               Join Operator
                 condition map:
@@ -272,7 +275,7 @@ STAGE PLANS:
                     Map-reduce partition columns: _col0 (type: int), _col1 (type: int)
                     Statistics: Num rows: 63350266 Data size: 4908414421 Basic stats: COMPLETE Column stats: NONE
                     value expressions: _col2 (type: decimal(17,2))
-        Reducer 3 
+        Reducer 4 
             Execution mode: vectorized
             Local Work:
               Map Reduce Local Work
@@ -283,54 +286,57 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 31675133 Data size: 2454207210 Basic stats: COMPLETE Column stats: NONE
-                Select Operator
-                  expressions: _col1 (type: int), _col0 (type: int), _col2 (type: decimal(17,2))
-                  outputColumnNames: _col0, _col1, _col2
+                Filter Operator
+                  predicate: _col2 is not null (type: boolean)
                   Statistics: Num rows: 31675133 Data size: 2454207210 Basic stats: COMPLETE Column stats: NONE
-                  Map Join Operator
-                    condition map:
-                         Inner Join 0 to 1
-                    keys:
-                      0 _col1 (type: int)
-                      1 _col0 (type: int)
+                  Select Operator
+                    expressions: _col1 (type: int), _col0 (type: int), _col2 (type: decimal(17,2))
                     outputColumnNames: _col0, _col1, _col2
-                    input vertices:
-                      1 Map 8
-                    Statistics: Num rows: 34842647 Data size: 2699627989 Basic stats: COMPLETE Column stats: NONE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Statistics: Num rows: 31675133 Data size: 2454207210 Basic stats: COMPLETE Column stats: NONE
+                    Map Join Operator
+                      condition map:
+                           Inner Join 0 to 1
+                      keys:
+                        0 _col0 (type: int)
+                        1 _col1 (type: int)
+                      outputColumnNames: _col1, _col2, _col3
+                      input vertices:
+                        0 Map 1
                       Statistics: Num rows: 34842647 Data size: 2699627989 Basic stats: COMPLETE Column stats: NONE
-                      value expressions: _col1 (type: int), _col2 (type: decimal(17,2))
-        Reducer 4 
+                      Reduce Output Operator
+                        key expressions: _col1 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col1 (type: int)
+                        Statistics: Num rows: 34842647 Data size: 2699627989 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col2 (type: int), _col3 (type: decimal(17,2))
+        Reducer 5 
             Reduce Operator Tree:
               Join Operator
                 condition map:
                      Inner Join 0 to 1
                 keys:
-                  0 _col0 (type: int)
+                  0 _col1 (type: int)
                   1 _col0 (type: int)
-                outputColumnNames: _col1, _col2, _col5
+                outputColumnNames: _col2, _col3, _col5
                 Statistics: Num rows: 88000001 Data size: 75681779077 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
-                  key expressions: _col1 (type: int)
+                  key expressions: _col2 (type: int)
                   sort order: +
-                  Map-reduce partition columns: _col1 (type: int)
+                  Map-reduce partition columns: _col2 (type: int)
                   Statistics: Num rows: 88000001 Data size: 75681779077 Basic stats: COMPLETE Column stats: NONE
-                  value expressions: _col2 (type: decimal(17,2)), _col5 (type: string)
-        Reducer 5 
+                  value expressions: _col3 (type: decimal(17,2)), _col5 (type: string)
+        Reducer 6 
             Reduce Operator Tree:
               Join Operator
                 condition map:
                      Inner Join 0 to 1
                 keys:
-                  0 _col1 (type: int)
+                  0 _col2 (type: int)
                   1 _col1 (type: int)
-                outputColumnNames: _col2, _col5, _col6
+                outputColumnNames: _col3, _col5, _col6
                 Statistics: Num rows: 96800003 Data size: 83249958789 Basic stats: COMPLETE Column stats: NONE
                 Filter Operator
-                  predicate: (_col2 > _col6) (type: boolean)
+                  predicate: (_col3 > _col6) (type: boolean)
                   Statistics: Num rows: 32266667 Data size: 27749985689 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: _col5 (type: string)
@@ -341,7 +347,7 @@ STAGE PLANS:
                       sort order: +
                       Statistics: Num rows: 32266667 Data size: 27749985689 Basic stats: COMPLETE Column stats: NONE
                       TopN Hash Memory Usage: 0.1
-        Reducer 6 
+        Reducer 7 
             Execution mode: vectorized
             Reduce Operator Tree:
               Select Operator
diff --git a/ql/src/test/results/clientpositive/perf/spark/query23.q.out b/ql/src/test/results/clientpositive/perf/spark/query23.q.out
index 36bd0e0..3b9cb56 100644
--- a/ql/src/test/results/clientpositive/perf/spark/query23.q.out
+++ b/ql/src/test/results/clientpositive/perf/spark/query23.q.out
@@ -1,5 +1,5 @@
-Warning: Map Join MAPJOIN[204][bigTable=?] in task 'Stage-1:MAPRED' is a cross product
-Warning: Map Join MAPJOIN[205][bigTable=?] in task 'Stage-1:MAPRED' is a cross product
+Warning: Map Join MAPJOIN[210][bigTable=?] in task 'Stage-1:MAPRED' is a cross product
+Warning: Map Join MAPJOIN[211][bigTable=?] in task 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: explain
 with frequent_ss_items as 
  (select substr(i_item_desc,1,30) itemdesc,i_item_sk item_sk,d_date solddate,count(*) cnt
@@ -262,14 +262,17 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 224 Basic stats: COMPLETE Column stats: NONE
-                Select Operator
-                  expressions: (0.95 * _col0) (type: decimal(31,4))
-                  outputColumnNames: _col0
+                Filter Operator
+                  predicate: _col0 is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 224 Basic stats: COMPLETE Column stats: NONE
-                  Spark HashTable Sink Operator
-                    keys:
-                      0 
-                      1 
+                  Select Operator
+                    expressions: (0.95 * _col0) (type: decimal(31,4))
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 Data size: 224 Basic stats: COMPLETE Column stats: NONE
+                    Spark HashTable Sink Operator
+                      keys:
+                        0 
+                        1 
 
   Stage: Stage-3
     Spark
@@ -410,14 +413,17 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 224 Basic stats: COMPLETE Column stats: NONE
-                Select Operator
-                  expressions: (0.95 * _col0) (type: decimal(31,4))
-                  outputColumnNames: _col0
+                Filter Operator
+                  predicate: _col0 is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 224 Basic stats: COMPLETE Column stats: NONE
-                  Spark HashTable Sink Operator
-                    keys:
-                      0 
-                      1 
+                  Select Operator
+                    expressions: (0.95 * _col0) (type: decimal(31,4))
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 Data size: 224 Basic stats: COMPLETE Column stats: NONE
+                    Spark HashTable Sink Operator
+                      keys:
+                        0 
+                        1 
 
   Stage: Stage-1
     Spark
@@ -659,34 +665,37 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 316797606 Data size: 27947976754 Basic stats: COMPLETE Column stats: NONE
-                Map Join Operator
-                  condition map:
-                       Inner Join 0 to 1
-                  keys:
-                    0 
-                    1 
-                  outputColumnNames: _col0, _col1, _col2
-                  input vertices:
-                    1 Reducer 21
-                  Statistics: Num rows: 316797606 Data size: 99227438104 Basic stats: COMPLETE Column stats: NONE
-                  Filter Operator
-                    predicate: (_col1 > _col2) (type: boolean)
-                    Statistics: Num rows: 105599202 Data size: 33075812701 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: _col0 (type: int)
-                      outputColumnNames: _col0
+                Filter Operator
+                  predicate: _col1 is not null (type: boolean)
+                  Statistics: Num rows: 316797606 Data size: 27947976754 Basic stats: COMPLETE Column stats: NONE
+                  Map Join Operator
+                    condition map:
+                         Inner Join 0 to 1
+                    keys:
+                      0 
+                      1 
+                    outputColumnNames: _col0, _col1, _col2
+                    input vertices:
+                      1 Reducer 21
+                    Statistics: Num rows: 316797606 Data size: 99227438104 Basic stats: COMPLETE Column stats: NONE
+                    Filter Operator
+                      predicate: (_col1 > _col2) (type: boolean)
                       Statistics: Num rows: 105599202 Data size: 33075812701 Basic stats: COMPLETE Column stats: NONE
-                      Group By Operator
-                        keys: _col0 (type: int)
-                        minReductionHashAggr: 0.99
-                        mode: hash
+                      Select Operator
+                        expressions: _col0 (type: int)
                         outputColumnNames: _col0
                         Statistics: Num rows: 105599202 Data size: 33075812701 Basic stats: COMPLETE Column stats: NONE
-                        Reduce Output Operator
-                          key expressions: _col0 (type: int)
-                          sort order: +
-                          Map-reduce partition columns: _col0 (type: int)
+                        Group By Operator
+                          keys: _col0 (type: int)
+                          minReductionHashAggr: 0.99
+                          mode: hash
+                          outputColumnNames: _col0
                           Statistics: Num rows: 105599202 Data size: 33075812701 Basic stats: COMPLETE Column stats: NONE
+                          Reduce Output Operator
+                            key expressions: _col0 (type: int)
+                            sort order: +
+                            Map-reduce partition columns: _col0 (type: int)
+                            Statistics: Num rows: 105599202 Data size: 33075812701 Basic stats: COMPLETE Column stats: NONE
         Reducer 2 
             Reduce Operator Tree:
               Join Operator
@@ -822,34 +831,37 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 316797606 Data size: 27947976754 Basic stats: COMPLETE Column stats: NONE
-                Map Join Operator
-                  condition map:
-                       Inner Join 0 to 1
-                  keys:
-                    0 
-                    1 
-                  outputColumnNames: _col0, _col1, _col2
-                  input vertices:
-                    1 Reducer 43
-                  Statistics: Num rows: 316797606 Data size: 99227438104 Basic stats: COMPLETE Column stats: NONE
-                  Filter Operator
-                    predicate: (_col1 > _col2) (type: boolean)
-                    Statistics: Num rows: 105599202 Data size: 33075812701 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: _col0 (type: int)
-                      outputColumnNames: _col0
+                Filter Operator
+                  predicate: _col1 is not null (type: boolean)
+                  Statistics: Num rows: 316797606 Data size: 27947976754 Basic stats: COMPLETE Column stats: NONE
+                  Map Join Operator
+                    condition map:
+                         Inner Join 0 to 1
+                    keys:
+                      0 
+                      1 
+                    outputColumnNames: _col0, _col1, _col2
+                    input vertices:
+                      1 Reducer 43
+                    Statistics: Num rows: 316797606 Data size: 99227438104 Basic stats: COMPLETE Column stats: NONE
+                    Filter Operator
+                      predicate: (_col1 > _col2) (type: boolean)
                       Statistics: Num rows: 105599202 Data size: 33075812701 Basic stats: COMPLETE Column stats: NONE
-                      Group By Operator
-                        keys: _col0 (type: int)
-                        minReductionHashAggr: 0.99
-                        mode: hash
+                      Select Operator
+                        expressions: _col0 (type: int)
                         outputColumnNames: _col0
                         Statistics: Num rows: 105599202 Data size: 33075812701 Basic stats: COMPLETE Column stats: NONE
-                        Reduce Output Operator
-                          key expressions: _col0 (type: int)
-                          sort order: +
-                          Map-reduce partition columns: _col0 (type: int)
+                        Group By Operator
+                          keys: _col0 (type: int)
+                          minReductionHashAggr: 0.99
+                          mode: hash
+                          outputColumnNames: _col0
                           Statistics: Num rows: 105599202 Data size: 33075812701 Basic stats: COMPLETE Column stats: NONE
+                          Reduce Output Operator
+                            key expressions: _col0 (type: int)
+                            sort order: +
+                            Map-reduce partition columns: _col0 (type: int)
+                            Statistics: Num rows: 105599202 Data size: 33075812701 Basic stats: COMPLETE Column stats: NONE
         Reducer 4 
             Reduce Operator Tree:
               Join Operator
diff --git a/ql/src/test/results/clientpositive/perf/spark/query24.q.out b/ql/src/test/results/clientpositive/perf/spark/query24.q.out
index 1bf3c75..33f7ada 100644
--- a/ql/src/test/results/clientpositive/perf/spark/query24.q.out
+++ b/ql/src/test/results/clientpositive/perf/spark/query24.q.out
@@ -1,4 +1,4 @@
-Warning: Map Join MAPJOIN[107][bigTable=?] in task 'Stage-1:MAPRED' is a cross product
+Warning: Map Join MAPJOIN[111][bigTable=?] in task 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: explain
 with ssales as
 (select c_last_name
@@ -375,14 +375,17 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 232 Basic stats: COMPLETE Column stats: NONE
-                Select Operator
-                  expressions: (0.05 * (_col0 / _col1)) (type: decimal(38,12))
-                  outputColumnNames: _col0
+                Filter Operator
+                  predicate: (_col0 is not null and _col1 is not null) (type: boolean)
                   Statistics: Num rows: 1 Data size: 232 Basic stats: COMPLETE Column stats: NONE
-                  Spark HashTable Sink Operator
-                    keys:
-                      0 
-                      1 
+                  Select Operator
+                    expressions: (0.05 * (_col0 / _col1)) (type: decimal(38,12))
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 Data size: 232 Basic stats: COMPLETE Column stats: NONE
+                    Spark HashTable Sink Operator
+                      keys:
+                        0 
+                        1 
 
   Stage: Stage-4
     Spark
@@ -610,30 +613,33 @@ STAGE PLANS:
                       expressions: _col1 (type: string), _col0 (type: string), _col2 (type: string), _col3 (type: decimal(27,2))
                       outputColumnNames: _col0, _col1, _col2, _col3
                       Statistics: Num rows: 191662559 Data size: 16908526602 Basic stats: COMPLETE Column stats: NONE
-                      Map Join Operator
-                        condition map:
-                             Inner Join 0 to 1
-                        keys:
-                          0 
-                          1 
-                        outputColumnNames: _col0, _col1, _col2, _col3, _col4
-                        input vertices:
-                          1 Reducer 18
-                        Statistics: Num rows: 191662559 Data size: 61565902849 Basic stats: COMPLETE Column stats: NONE
-                        Filter Operator
-                          predicate: (_col3 > _col4) (type: boolean)
-                          Statistics: Num rows: 63887519 Data size: 20521967402 Basic stats: COMPLETE Column stats: NONE
-                          Select Operator
-                            expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: decimal(27,2))
-                            outputColumnNames: _col0, _col1, _col2, _col3
+                      Filter Operator
+                        predicate: _col3 is not null (type: boolean)
+                        Statistics: Num rows: 191662559 Data size: 16908526602 Basic stats: COMPLETE Column stats: NONE
+                        Map Join Operator
+                          condition map:
+                               Inner Join 0 to 1
+                          keys:
+                            0 
+                            1 
+                          outputColumnNames: _col0, _col1, _col2, _col3, _col4
+                          input vertices:
+                            1 Reducer 18
+                          Statistics: Num rows: 191662559 Data size: 61565902849 Basic stats: COMPLETE Column stats: NONE
+                          Filter Operator
+                            predicate: (_col3 > _col4) (type: boolean)
                             Statistics: Num rows: 63887519 Data size: 20521967402 Basic stats: COMPLETE Column stats: NONE
-                            File Output Operator
-                              compressed: false
+                            Select Operator
+                              expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: decimal(27,2))
+                              outputColumnNames: _col0, _col1, _col2, _col3
                               Statistics: Num rows: 63887519 Data size: 20521967402 Basic stats: COMPLETE Column stats: NONE
-                              table:
-                                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                              File Output Operator
+                                compressed: false
+                                Statistics: Num rows: 63887519 Data size: 20521967402 Basic stats: COMPLETE Column stats: NONE
+                                table:
+                                    input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                                    output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
         Reducer 8 
             Reduce Operator Tree:
               Join Operator
diff --git a/ql/src/test/results/clientpositive/perf/spark/query30.q.out b/ql/src/test/results/clientpositive/perf/spark/query30.q.out
index 2a3a9a5..83e8c77 100644
--- a/ql/src/test/results/clientpositive/perf/spark/query30.q.out
+++ b/ql/src/test/results/clientpositive/perf/spark/query30.q.out
@@ -326,16 +326,19 @@ STAGE PLANS:
                     mode: complete
                     outputColumnNames: _col0, _col1, _col2
                     Statistics: Num rows: 11000000 Data size: 11163678945 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: ((_col1 / _col2) * 1.2) (type: decimal(38,11)), _col0 (type: string)
-                      outputColumnNames: _col0, _col1
+                    Filter Operator
+                      predicate: (_col1 is not null and _col2 is not null) (type: boolean)
                       Statistics: Num rows: 11000000 Data size: 11163678945 Basic stats: COMPLETE Column stats: NONE
-                      Reduce Output Operator
-                        key expressions: _col1 (type: string)
-                        sort order: +
-                        Map-reduce partition columns: _col1 (type: string)
+                      Select Operator
+                        expressions: ((_col1 / _col2) * 1.2) (type: decimal(38,11)), _col0 (type: string)
+                        outputColumnNames: _col0, _col1
                         Statistics: Num rows: 11000000 Data size: 11163678945 Basic stats: COMPLETE Column stats: NONE
-                        value expressions: _col0 (type: decimal(38,11))
+                        Reduce Output Operator
+                          key expressions: _col1 (type: string)
+                          sort order: +
+                          Map-reduce partition columns: _col1 (type: string)
+                          Statistics: Num rows: 11000000 Data size: 11163678945 Basic stats: COMPLETE Column stats: NONE
+                          value expressions: _col0 (type: decimal(38,11))
         Reducer 2 
             Reduce Operator Tree:
               Join Operator
@@ -440,12 +443,15 @@ STAGE PLANS:
                   expressions: _col1 (type: int), _col0 (type: string), _col2 (type: decimal(17,2))
                   outputColumnNames: _col0, _col1, _col2
                   Statistics: Num rows: 22000000 Data size: 22327357890 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col1 (type: string)
-                    sort order: +
-                    Map-reduce partition columns: _col1 (type: string)
+                  Filter Operator
+                    predicate: _col2 is not null (type: boolean)
                     Statistics: Num rows: 22000000 Data size: 22327357890 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col0 (type: int), _col2 (type: decimal(17,2))
+                    Reduce Output Operator
+                      key expressions: _col1 (type: string)
+                      sort order: +
+                      Map-reduce partition columns: _col1 (type: string)
+                      Statistics: Num rows: 22000000 Data size: 22327357890 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col0 (type: int), _col2 (type: decimal(17,2))
 
   Stage: Stage-0
     Fetch Operator
diff --git a/ql/src/test/results/clientpositive/perf/spark/query32.q.out b/ql/src/test/results/clientpositive/perf/spark/query32.q.out
index 5370e95..e3e002e 100644
--- a/ql/src/test/results/clientpositive/perf/spark/query32.q.out
+++ b/ql/src/test/results/clientpositive/perf/spark/query32.q.out
@@ -120,20 +120,19 @@ STAGE PLANS:
   Stage: Stage-1
     Spark
       Edges:
-        Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 520), Reducer 7 (PARTITION-LEVEL SORT, 520)
+        Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 506), Map 5 (PARTITION-LEVEL SORT, 506), Reducer 7 (PARTITION-LEVEL SORT, 506)
         Reducer 3 <- Reducer 2 (GROUP, 1)
-        Reducer 6 <- Map 5 (GROUP, 336)
-        Reducer 7 <- Map 9 (PARTITION-LEVEL SORT, 171), Reducer 6 (PARTITION-LEVEL SORT, 171)
+        Reducer 7 <- Map 6 (GROUP, 336)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: catalog_sales
-                  filterExpr: (cs_item_sk is not null and cs_sold_date_sk is not null) (type: boolean)
+                  filterExpr: (cs_item_sk is not null and cs_sold_date_sk is not null and cs_ext_discount_amt is not null) (type: boolean)
                   Statistics: Num rows: 287989836 Data size: 38999608952 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: (cs_item_sk is not null and cs_sold_date_sk is not null) (type: boolean)
+                    predicate: (cs_ext_discount_amt is not null and cs_item_sk is not null and cs_sold_date_sk is not null) (type: boolean)
                     Statistics: Num rows: 287989836 Data size: 38999608952 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: cs_sold_date_sk (type: int), cs_item_sk (type: int), cs_ext_discount_amt (type: decimal(7,2))
@@ -161,6 +160,25 @@ STAGE PLANS:
         Map 5 
             Map Operator Tree:
                 TableScan
+                  alias: item
+                  filterExpr: ((i_manufact_id = 269) and i_item_sk is not null) (type: boolean)
+                  Statistics: Num rows: 462000 Data size: 663560457 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: ((i_manufact_id = 269) and i_item_sk is not null) (type: boolean)
+                    Statistics: Num rows: 231000 Data size: 331780228 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: i_item_sk (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 231000 Data size: 331780228 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 231000 Data size: 331780228 Basic stats: COMPLETE Column stats: NONE
+            Execution mode: vectorized
+        Map 6 
+            Map Operator Tree:
+                TableScan
                   alias: catalog_sales
                   filterExpr: (cs_item_sk is not null and cs_sold_date_sk is not null) (type: boolean)
                   Statistics: Num rows: 287989836 Data size: 38999608952 Basic stats: COMPLETE Column stats: NONE
@@ -197,42 +215,25 @@ STAGE PLANS:
             Execution mode: vectorized
             Local Work:
               Map Reduce Local Work
-        Map 9 
-            Map Operator Tree:
-                TableScan
-                  alias: item
-                  filterExpr: ((i_manufact_id = 269) and i_item_sk is not null) (type: boolean)
-                  Statistics: Num rows: 462000 Data size: 663560457 Basic stats: COMPLETE Column stats: NONE
-                  Filter Operator
-                    predicate: ((i_manufact_id = 269) and i_item_sk is not null) (type: boolean)
-                    Statistics: Num rows: 231000 Data size: 331780228 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: i_item_sk (type: int)
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 231000 Data size: 331780228 Basic stats: COMPLETE Column stats: NONE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: int)
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 231000 Data size: 331780228 Basic stats: COMPLETE Column stats: NONE
-            Execution mode: vectorized
         Reducer 2 
             Reduce Operator Tree:
               Join Operator
                 condition map:
                      Inner Join 0 to 1
+                     Inner Join 1 to 2
                 keys:
                   0 _col1 (type: int)
-                  1 _col2 (type: int)
-                outputColumnNames: _col2, _col5
-                Statistics: Num rows: 348467716 Data size: 47189528877 Basic stats: COMPLETE Column stats: NONE
+                  1 _col0 (type: int)
+                  2 _col0 (type: int)
+                outputColumnNames: _col2, _col6
+                Statistics: Num rows: 696935432 Data size: 94379057755 Basic stats: COMPLETE Column stats: NONE
                 Filter Operator
-                  predicate: (_col2 > _col5) (type: boolean)
-                  Statistics: Num rows: 116155905 Data size: 15729842913 Basic stats: COMPLETE Column stats: NONE
+                  predicate: (_col2 > _col6) (type: boolean)
+                  Statistics: Num rows: 232311810 Data size: 31459685828 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: _col2 (type: decimal(7,2))
                     outputColumnNames: _col2
-                    Statistics: Num rows: 116155905 Data size: 15729842913 Basic stats: COMPLETE Column stats: NONE
+                    Statistics: Num rows: 232311810 Data size: 31459685828 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
                       aggregations: sum(_col2)
                       minReductionHashAggr: 0.99
@@ -258,7 +259,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 7 
             Execution mode: vectorized
             Reduce Operator Tree:
               Group By Operator
@@ -267,32 +268,19 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 158394413 Data size: 21449785388 Basic stats: COMPLETE Column stats: NONE
-                Select Operator
-                  expressions: _col0 (type: int), CAST( (1.3 * (_col1 / _col2)) AS decimal(14,7)) (type: decimal(14,7))
-                  outputColumnNames: _col0, _col1
+                Filter Operator
+                  predicate: CAST( (1.3 * (_col1 / _col2)) AS decimal(14,7)) is not null (type: boolean)
                   Statistics: Num rows: 158394413 Data size: 21449785388 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: int)
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: int)
+                  Select Operator
+                    expressions: _col0 (type: int), CAST( (1.3 * (_col1 / _col2)) AS decimal(14,7)) (type: decimal(14,7))
+                    outputColumnNames: _col0, _col1
                     Statistics: Num rows: 158394413 Data size: 21449785388 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col1 (type: decimal(14,7))
-        Reducer 7 
-            Reduce Operator Tree:
-              Join Operator
-                condition map:
-                     Inner Join 0 to 1
-                keys:
-                  0 _col0 (type: int)
-                  1 _col0 (type: int)
-                outputColumnNames: _col1, _col2
-                Statistics: Num rows: 174233858 Data size: 23594764438 Basic stats: COMPLETE Column stats: NONE
-                Reduce Output Operator
-                  key expressions: _col2 (type: int)
-                  sort order: +
-                  Map-reduce partition columns: _col2 (type: int)
-                  Statistics: Num rows: 174233858 Data size: 23594764438 Basic stats: COMPLETE Column stats: NONE
-                  value expressions: _col1 (type: decimal(14,7))
+                    Reduce Output Operator
+                      key expressions: _col0 (type: int)
+                      sort order: +
+                      Map-reduce partition columns: _col0 (type: int)
+                      Statistics: Num rows: 158394413 Data size: 21449785388 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col1 (type: decimal(14,7))
 
   Stage: Stage-0
     Fetch Operator
diff --git a/ql/src/test/results/clientpositive/perf/spark/query44.q.out b/ql/src/test/results/clientpositive/perf/spark/query44.q.out
index ccd240b..3312880 100644
--- a/ql/src/test/results/clientpositive/perf/spark/query44.q.out
+++ b/ql/src/test/results/clientpositive/perf/spark/query44.q.out
@@ -1,5 +1,5 @@
-Warning: Shuffle Join JOIN[20][tables = [$hdt$_1, $hdt$_2]] in Work 'Reducer 7' is a cross product
-Warning: Shuffle Join JOIN[49][tables = [$hdt$_3, $hdt$_4]] in Work 'Reducer 15' is a cross product
+Warning: Shuffle Join JOIN[38][tables = [$hdt$_2, $hdt$_3, $hdt$_1]] in Work 'Reducer 8' is a cross product
+Warning: Shuffle Join JOIN[85][tables = [$hdt$_4, $hdt$_5, $hdt$_3]] in Work 'Reducer 19' is a cross product
 PREHOOK: query: explain
 select  asceding.rnk, i1.i_product_name best_performing, i2.i_product_name worst_performing
 from(select *
@@ -82,18 +82,20 @@ STAGE PLANS:
   Stage: Stage-1
     Spark
       Edges:
-        Reducer 10 <- Map 17 (GROUP, 100)
-        Reducer 12 <- Map 11 (PARTITION-LEVEL SORT, 1009), Reducer 16 (PARTITION-LEVEL SORT, 1009)
-        Reducer 14 <- Map 13 (GROUP, 199)
-        Reducer 15 <- Reducer 14 (PARTITION-LEVEL SORT, 1), Reducer 18 (PARTITION-LEVEL SORT, 1)
-        Reducer 16 <- Reducer 15 (PARTITION-LEVEL SORT, 1009)
-        Reducer 18 <- Map 17 (GROUP, 100)
-        Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1009), Reducer 8 (PARTITION-LEVEL SORT, 1009)
-        Reducer 3 <- Reducer 12 (PARTITION-LEVEL SORT, 1009), Reducer 2 (PARTITION-LEVEL SORT, 1009)
+        Reducer 11 <- Map 10 (GROUP, 100)
+        Reducer 13 <- Map 12 (GROUP, 199)
+        Reducer 15 <- Map 14 (PARTITION-LEVEL SORT, 1009), Reducer 20 (PARTITION-LEVEL SORT, 1009)
+        Reducer 17 <- Map 16 (GROUP, 100)
+        Reducer 18 <- Reducer 17 (GROUP, 1)
+        Reducer 19 <- Reducer 18 (PARTITION-LEVEL SORT, 1), Reducer 22 (PARTITION-LEVEL SORT, 1), Reducer 24 (PARTITION-LEVEL SORT, 1)
+        Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1009), Reducer 9 (PARTITION-LEVEL SORT, 1009)
+        Reducer 20 <- Reducer 19 (PARTITION-LEVEL SORT, 1009)
+        Reducer 22 <- Map 10 (GROUP, 100)
+        Reducer 24 <- Map 12 (GROUP, 199)
+        Reducer 3 <- Reducer 15 (PARTITION-LEVEL SORT, 1009), Reducer 2 (PARTITION-LEVEL SORT, 1009)
         Reducer 4 <- Reducer 3 (SORT, 1)
-        Reducer 6 <- Map 13 (GROUP, 199)
-        Reducer 7 <- Reducer 10 (PARTITION-LEVEL SORT, 1), Reducer 6 (PARTITION-LEVEL SORT, 1)
-        Reducer 8 <- Reducer 7 (PARTITION-LEVEL SORT, 1009)
+        Reducer 8 <- Reducer 11 (PARTITION-LEVEL SORT, 1), Reducer 13 (PARTITION-LEVEL SORT, 1), Reducer 18 (PARTITION-LEVEL SORT, 1)
+        Reducer 9 <- Reducer 8 (PARTITION-LEVEL SORT, 1009)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -116,27 +118,34 @@ STAGE PLANS:
                         Statistics: Num rows: 462000 Data size: 663560457 Basic stats: COMPLETE Column stats: NONE
                         value expressions: _col1 (type: string)
             Execution mode: vectorized
-        Map 11 
+        Map 10 
             Map Operator Tree:
                 TableScan
-                  alias: i2
-                  filterExpr: i_item_sk is not null (type: boolean)
-                  Statistics: Num rows: 462000 Data size: 663560457 Basic stats: COMPLETE Column stats: NONE
+                  alias: store_sales
+                  filterExpr: ((ss_store_sk = 410) and ss_hdemo_sk is null) (type: boolean)
+                  Statistics: Num rows: 575995635 Data size: 50814502088 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: i_item_sk is not null (type: boolean)
-                    Statistics: Num rows: 462000 Data size: 663560457 Basic stats: COMPLETE Column stats: NONE
+                    predicate: ((ss_store_sk = 410) and ss_hdemo_sk is null) (type: boolean)
+                    Statistics: Num rows: 143998908 Data size: 12703625455 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      expressions: i_item_sk (type: int), i_product_name (type: string)
-                      outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 462000 Data size: 663560457 Basic stats: COMPLETE Column stats: NONE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: int)
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 462000 Data size: 663560457 Basic stats: COMPLETE Column stats: NONE
-                        value expressions: _col1 (type: string)
+                      expressions: ss_net_profit (type: decimal(7,2))
+                      outputColumnNames: _col1
+                      Statistics: Num rows: 143998908 Data size: 12703625455 Basic stats: COMPLETE Column stats: NONE
+                      Group By Operator
+                        aggregations: sum(_col1), count(_col1)
+                        keys: true (type: boolean)
+                        minReductionHashAggr: 0.99
+                        mode: hash
+                        outputColumnNames: _col0, _col1, _col2
+                        Statistics: Num rows: 143998908 Data size: 12703625455 Basic stats: COMPLETE Column stats: NONE
+                        Reduce Output Operator
+                          key expressions: _col0 (type: boolean)
+                          sort order: +
+                          Map-reduce partition columns: _col0 (type: boolean)
+                          Statistics: Num rows: 143998908 Data size: 12703625455 Basic stats: COMPLETE Column stats: NONE
+                          value expressions: _col1 (type: decimal(17,2)), _col2 (type: bigint)
             Execution mode: vectorized
-        Map 13 
+        Map 12 
             Map Operator Tree:
                 TableScan
                   alias: ss1
@@ -163,7 +172,27 @@ STAGE PLANS:
                           Statistics: Num rows: 287997817 Data size: 25407250999 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col1 (type: decimal(17,2)), _col2 (type: bigint)
             Execution mode: vectorized
-        Map 17 
+        Map 14 
+            Map Operator Tree:
+                TableScan
+                  alias: i2
+                  filterExpr: i_item_sk is not null (type: boolean)
+                  Statistics: Num rows: 462000 Data size: 663560457 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: i_item_sk is not null (type: boolean)
+                    Statistics: Num rows: 462000 Data size: 663560457 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: i_item_sk (type: int), i_product_name (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 462000 Data size: 663560457 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 462000 Data size: 663560457 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col1 (type: string)
+            Execution mode: vectorized
+        Map 16 
             Map Operator Tree:
                 TableScan
                   alias: store_sales
@@ -173,24 +202,20 @@ STAGE PLANS:
                     predicate: ((ss_store_sk = 410) and ss_hdemo_sk is null) (type: boolean)
                     Statistics: Num rows: 143998908 Data size: 12703625455 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      expressions: ss_net_profit (type: decimal(7,2))
-                      outputColumnNames: _col1
                       Statistics: Num rows: 143998908 Data size: 12703625455 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
-                        aggregations: sum(_col1), count(_col1)
                         keys: true (type: boolean)
                         minReductionHashAggr: 0.99
                         mode: hash
-                        outputColumnNames: _col0, _col1, _col2
+                        outputColumnNames: _col0
                         Statistics: Num rows: 143998908 Data size: 12703625455 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
                           key expressions: _col0 (type: boolean)
                           sort order: +
                           Map-reduce partition columns: _col0 (type: boolean)
                           Statistics: Num rows: 143998908 Data size: 12703625455 Basic stats: COMPLETE Column stats: NONE
-                          value expressions: _col1 (type: decimal(17,2)), _col2 (type: bigint)
             Execution mode: vectorized
-        Reducer 10 
+        Reducer 11 
             Execution mode: vectorized
             Reduce Operator Tree:
               Group By Operator
@@ -200,14 +225,41 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 71999454 Data size: 6351812727 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  expressions: (0.9 * (_col1 / _col2)) (type: decimal(38,22))
-                  outputColumnNames: _col0
+                  expressions: _col1 (type: decimal(17,2)), _col2 (type: bigint)
+                  outputColumnNames: _col1, _col2
                   Statistics: Num rows: 71999454 Data size: 6351812727 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    sort order: 
+                  Filter Operator
+                    predicate: (_col1 is not null and _col2 is not null) (type: boolean)
                     Statistics: Num rows: 71999454 Data size: 6351812727 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col0 (type: decimal(38,22))
-        Reducer 12 
+                    Select Operator
+                      expressions: (0.9 * (_col1 / _col2)) (type: decimal(38,22))
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 71999454 Data size: 6351812727 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        sort order: 
+                        Statistics: Num rows: 71999454 Data size: 6351812727 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col0 (type: decimal(38,22))
+        Reducer 13 
+            Execution mode: vectorized
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: sum(VALUE._col0), count(VALUE._col1)
+                keys: KEY._col0 (type: int)
+                mode: mergepartial
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 143998908 Data size: 12703625455 Basic stats: COMPLETE Column stats: NONE
+                Filter Operator
+                  predicate: (_col1 is not null and _col2 is not null) (type: boolean)
+                  Statistics: Num rows: 143998908 Data size: 12703625455 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: _col0 (type: int), (_col1 / _col2) (type: decimal(37,22))
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 143998908 Data size: 12703625455 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      sort order: 
+                      Statistics: Num rows: 143998908 Data size: 12703625455 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col0 (type: int), _col1 (type: decimal(37,22))
+        Reducer 15 
             Reduce Operator Tree:
               Join Operator
                 condition map:
@@ -216,92 +268,129 @@ STAGE PLANS:
                   0 _col0 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col1, _col3
-                Statistics: Num rows: 1267180808338276 Data size: 224849298143006048 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 1267180808338276 Data size: 234986744609712256 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col3 (type: int)
                   sort order: +
                   Map-reduce partition columns: _col3 (type: int)
-                  Statistics: Num rows: 1267180808338276 Data size: 224849298143006048 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 1267180808338276 Data size: 234986744609712256 Basic stats: COMPLETE Column stats: NONE
                   value expressions: _col1 (type: string)
-        Reducer 14 
+        Reducer 17 
             Execution mode: vectorized
             Reduce Operator Tree:
               Group By Operator
-                aggregations: sum(VALUE._col0), count(VALUE._col1)
-                keys: KEY._col0 (type: int)
+                keys: KEY._col0 (type: boolean)
                 mode: mergepartial
-                outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 143998908 Data size: 12703625455 Basic stats: COMPLETE Column stats: NONE
+                outputColumnNames: _col0
+                Statistics: Num rows: 71999454 Data size: 6351812727 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  expressions: _col0 (type: int), (_col1 / _col2) (type: decimal(37,22))
-                  outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 143998908 Data size: 12703625455 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    sort order: 
-                    Statistics: Num rows: 143998908 Data size: 12703625455 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col0 (type: int), _col1 (type: decimal(37,22))
-        Reducer 15 
+                  Statistics: Num rows: 71999454 Data size: 6351812727 Basic stats: COMPLETE Column stats: NONE
+                  Group By Operator
+                    aggregations: count()
+                    minReductionHashAggr: 0.99
+                    mode: hash
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      sort order: 
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col0 (type: bigint)
+        Reducer 18 
+            Execution mode: vectorized
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                Filter Operator
+                  predicate: (sq_count_check(_col0) <= 1) (type: boolean)
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      sort order: 
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+        Reducer 19 
             Reduce Operator Tree:
               Join Operator
                 condition map:
                      Inner Join 0 to 1
+                     Inner Join 0 to 2
                 keys:
                   0 
                   1 
-                outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 10367842752596232 Data size: 1839676035841599918 Basic stats: COMPLETE Column stats: NONE
+                  2 
+                outputColumnNames: _col1, _col2, _col3
+                Statistics: Num rows: 10367842752596232 Data size: 1922618777862369774 Basic stats: COMPLETE Column stats: NONE
                 Filter Operator
-                  predicate: (_col1 > _col2) (type: boolean)
-                  Statistics: Num rows: 3455947584198744 Data size: 613225345280533248 Basic stats: COMPLETE Column stats: NONE
+                  predicate: (_col3 > _col1) (type: boolean)
+                  Statistics: Num rows: 3455947584198744 Data size: 640872925954123264 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
-                    key expressions: 0 (type: int), _col1 (type: decimal(37,22))
+                    key expressions: 0 (type: int), _col3 (type: decimal(37,22))
                     sort order: +-
                     Map-reduce partition columns: 0 (type: int)
-                    Statistics: Num rows: 3455947584198744 Data size: 613225345280533248 Basic stats: COMPLETE Column stats: NONE
+                    Statistics: Num rows: 3455947584198744 Data size: 640872925954123264 Basic stats: COMPLETE Column stats: NONE
                     TopN Hash Memory Usage: 0.1
-                    value expressions: _col0 (type: int)
-        Reducer 16 
+                    value expressions: _col2 (type: int)
+        Reducer 2 
+            Reduce Operator Tree:
+              Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: int)
+                  1 _col0 (type: int)
+                outputColumnNames: _col1, _col3
+                Statistics: Num rows: 1267180808338276 Data size: 234986744609712256 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col3 (type: int)
+                  sort order: +
+                  Map-reduce partition columns: _col3 (type: int)
+                  Statistics: Num rows: 1267180808338276 Data size: 234986744609712256 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col1 (type: string)
+        Reducer 20 
             Execution mode: vectorized
             Reduce Operator Tree:
               Select Operator
-                expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: decimal(37,22))
-                outputColumnNames: _col0, _col1
-                Statistics: Num rows: 3455947584198744 Data size: 613225345280533248 Basic stats: COMPLETE Column stats: NONE
+                expressions: VALUE._col2 (type: int), KEY.reducesinkkey1 (type: decimal(37,22))
+                outputColumnNames: _col2, _col3
+                Statistics: Num rows: 3455947584198744 Data size: 640872925954123264 Basic stats: COMPLETE Column stats: NONE
                 PTF Operator
                   Function definitions:
                       Input definition
                         input alias: ptf_0
-                        output shape: _col0: int, _col1: decimal(37,22)
+                        output shape: _col2: int, _col3: decimal(37,22)
                         type: WINDOWING
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1 DESC NULLS LAST
+                        order by: _col3 DESC NULLS LAST
                         partition by: 0
                         raw input shape:
                         window functions:
                             window function definition
                               alias: rank_window_0
-                              arguments: _col1
+                              arguments: _col3
                               name: rank
                               window function: GenericUDAFRankEvaluator
                               window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX)
                               isPivotResult: true
-                  Statistics: Num rows: 3455947584198744 Data size: 613225345280533248 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 3455947584198744 Data size: 640872925954123264 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: ((rank_window_0 < 11) and _col0 is not null) (type: boolean)
-                    Statistics: Num rows: 1151982528066248 Data size: 204408448426844416 Basic stats: COMPLETE Column stats: NONE
+                    predicate: ((rank_window_0 < 11) and _col2 is not null) (type: boolean)
+                    Statistics: Num rows: 1151982528066248 Data size: 213624308651374400 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      expressions: _col0 (type: int), rank_window_0 (type: int)
+                      expressions: _col2 (type: int), rank_window_0 (type: int)
                       outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 1151982528066248 Data size: 204408448426844416 Basic stats: COMPLETE Column stats: NONE
+                      Statistics: Num rows: 1151982528066248 Data size: 213624308651374400 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: int)
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 1151982528066248 Data size: 204408448426844416 Basic stats: COMPLETE Column stats: NONE
+                        Statistics: Num rows: 1151982528066248 Data size: 213624308651374400 Basic stats: COMPLETE Column stats: NONE
                         value expressions: _col1 (type: int)
-        Reducer 18 
+        Reducer 22 
             Execution mode: vectorized
             Reduce Operator Tree:
               Group By Operator
@@ -311,29 +400,40 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 71999454 Data size: 6351812727 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  expressions: (0.9 * (_col1 / _col2)) (type: decimal(38,22))
-                  outputColumnNames: _col0
+                  expressions: _col1 (type: decimal(17,2)), _col2 (type: bigint)
+                  outputColumnNames: _col1, _col2
                   Statistics: Num rows: 71999454 Data size: 6351812727 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    sort order: 
+                  Filter Operator
+                    predicate: (_col1 is not null and _col2 is not null) (type: boolean)
                     Statistics: Num rows: 71999454 Data size: 6351812727 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col0 (type: decimal(38,22))
-        Reducer 2 
+                    Select Operator
+                      expressions: (0.9 * (_col1 / _col2)) (type: decimal(38,22))
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 71999454 Data size: 6351812727 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        sort order: 
+                        Statistics: Num rows: 71999454 Data size: 6351812727 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col0 (type: decimal(38,22))
+        Reducer 24 
+            Execution mode: vectorized
             Reduce Operator Tree:
-              Join Operator
-                condition map:
-                     Inner Join 0 to 1
-                keys:
-                  0 _col0 (type: int)
-                  1 _col0 (type: int)
-                outputColumnNames: _col1, _col3
-                Statistics: Num rows: 1267180808338276 Data size: 224849298143006048 Basic stats: COMPLETE Column stats: NONE
-                Reduce Output Operator
-                  key expressions: _col3 (type: int)
-                  sort order: +
-                  Map-reduce partition columns: _col3 (type: int)
-                  Statistics: Num rows: 1267180808338276 Data size: 224849298143006048 Basic stats: COMPLETE Column stats: NONE
-                  value expressions: _col1 (type: string)
+              Group By Operator
+                aggregations: sum(VALUE._col0), count(VALUE._col1)
+                keys: KEY._col0 (type: int)
+                mode: mergepartial
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 143998908 Data size: 12703625455 Basic stats: COMPLETE Column stats: NONE
+                Filter Operator
+                  predicate: (_col1 is not null and _col2 is not null) (type: boolean)
+                  Statistics: Num rows: 143998908 Data size: 12703625455 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: _col0 (type: int), (_col1 / _col2) (type: decimal(37,22))
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 143998908 Data size: 12703625455 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      sort order: 
+                      Statistics: Num rows: 143998908 Data size: 12703625455 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col0 (type: int), _col1 (type: decimal(37,22))
         Reducer 3 
             Reduce Operator Tree:
               Join Operator
@@ -343,15 +443,15 @@ STAGE PLANS:
                   0 _col3 (type: int)
                   1 _col3 (type: int)
                 outputColumnNames: _col1, _col3, _col5
-                Statistics: Num rows: 1393898919384048 Data size: 247334233318131680 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 1393898919384048 Data size: 258485424673204064 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
                   expressions: _col3 (type: int), _col1 (type: string), _col5 (type: string)
                   outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 1393898919384048 Data size: 247334233318131680 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 1393898919384048 Data size: 258485424673204064 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col0 (type: int)
                     sort order: +
-                    Statistics: Num rows: 1393898919384048 Data size: 247334233318131680 Basic stats: COMPLETE Column stats: NONE
+                    Statistics: Num rows: 1393898919384048 Data size: 258485424673204064 Basic stats: COMPLETE Column stats: NONE
                     TopN Hash Memory Usage: 0.1
                     value expressions: _col1 (type: string), _col2 (type: string)
         Reducer 4 
@@ -360,94 +460,79 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string), VALUE._col1 (type: string)
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 1393898919384048 Data size: 247334233318131680 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 1393898919384048 Data size: 258485424673204064 Basic stats: COMPLETE Column stats: NONE
                 Limit
                   Number of rows: 100
-                  Statistics: Num rows: 100 Data size: 17700 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 100 Data size: 18500 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 100 Data size: 17700 Basic stats: COMPLETE Column stats: NONE
+                    Statistics: Num rows: 100 Data size: 18500 Basic stats: COMPLETE Column stats: NONE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                         serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 6 
-            Execution mode: vectorized
-            Reduce Operator Tree:
-              Group By Operator
-                aggregations: sum(VALUE._col0), count(VALUE._col1)
-                keys: KEY._col0 (type: int)
-                mode: mergepartial
-                outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 143998908 Data size: 12703625455 Basic stats: COMPLETE Column stats: NONE
-                Select Operator
-                  expressions: _col0 (type: int), (_col1 / _col2) (type: decimal(37,22))
-                  outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 143998908 Data size: 12703625455 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    sort order: 
-                    Statistics: Num rows: 143998908 Data size: 12703625455 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col0 (type: int), _col1 (type: decimal(37,22))
-        Reducer 7 
+        Reducer 8 
             Reduce Operator Tree:
               Join Operator
                 condition map:
                      Inner Join 0 to 1
+                     Inner Join 0 to 2
                 keys:
                   0 
                   1 
-                outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 10367842752596232 Data size: 1839676035841599918 Basic stats: COMPLETE Column stats: NONE
+                  2 
+                outputColumnNames: _col1, _col2, _col3
+                Statistics: Num rows: 10367842752596232 Data size: 1922618777862369774 Basic stats: COMPLETE Column stats: NONE
                 Filter Operator
-                  predicate: (_col1 > _col2) (type: boolean)
-                  Statistics: Num rows: 3455947584198744 Data size: 613225345280533248 Basic stats: COMPLETE Column stats: NONE
+                  predicate: (_col3 > _col1) (type: boolean)
+                  Statistics: Num rows: 3455947584198744 Data size: 640872925954123264 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
-                    key expressions: 0 (type: int), _col1 (type: decimal(37,22))
+                    key expressions: 0 (type: int), _col3 (type: decimal(37,22))
                     sort order: ++
                     Map-reduce partition columns: 0 (type: int)
-                    Statistics: Num rows: 3455947584198744 Data size: 613225345280533248 Basic stats: COMPLETE Column stats: NONE
+                    Statistics: Num rows: 3455947584198744 Data size: 640872925954123264 Basic stats: COMPLETE Column stats: NONE
                     TopN Hash Memory Usage: 0.1
-                    value expressions: _col0 (type: int)
-        Reducer 8 
+                    value expressions: _col2 (type: int)
+        Reducer 9 
             Execution mode: vectorized
             Reduce Operator Tree:
               Select Operator
-                expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: decimal(37,22))
-                outputColumnNames: _col0, _col1
-                Statistics: Num rows: 3455947584198744 Data size: 613225345280533248 Basic stats: COMPLETE Column stats: NONE
+                expressions: VALUE._col2 (type: int), KEY.reducesinkkey1 (type: decimal(37,22))
+                outputColumnNames: _col2, _col3
+                Statistics: Num rows: 3455947584198744 Data size: 640872925954123264 Basic stats: COMPLETE Column stats: NONE
                 PTF Operator
                   Function definitions:
                       Input definition
                         input alias: ptf_0
-                        output shape: _col0: int, _col1: decimal(37,22)
+                        output shape: _col2: int, _col3: decimal(37,22)
                         type: WINDOWING
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1 ASC NULLS FIRST
+                        order by: _col3 ASC NULLS FIRST
                         partition by: 0
                         raw input shape:
                         window functions:
                             window function definition
                               alias: rank_window_0
-                              arguments: _col1
+                              arguments: _col3
                               name: rank
                               window function: GenericUDAFRankEvaluator
                               window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX)
                               isPivotResult: true
-                  Statistics: Num rows: 3455947584198744 Data size: 613225345280533248 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 3455947584198744 Data size: 640872925954123264 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: ((rank_window_0 < 11) and _col0 is not null) (type: boolean)
-                    Statistics: Num rows: 1151982528066248 Data size: 204408448426844416 Basic stats: COMPLETE Column stats: NONE
+                    predicate: ((rank_window_0 < 11) and _col2 is not null) (type: boolean)
+                    Statistics: Num rows: 1151982528066248 Data size: 213624308651374400 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      expressions: _col0 (type: int), rank_window_0 (type: int)
+                      expressions: _col2 (type: int), rank_window_0 (type: int)
                       outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 1151982528066248 Data size: 204408448426844416 Basic stats: COMPLETE Column stats: NONE
+                      Statistics: Num rows: 1151982528066248 Data size: 213624308651374400 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: int)
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 1151982528066248 Data size: 204408448426844416 Basic stats: COMPLETE Column stats: NONE
+                        Statistics: Num rows: 1151982528066248 Data size: 213624308651374400 Basic stats: COMPLETE Column stats: NONE
                         value expressions: _col1 (type: int)
 
   Stage: Stage-0
diff --git a/ql/src/test/results/clientpositive/perf/spark/query54.q.out b/ql/src/test/results/clientpositive/perf/spark/query54.q.out
index 10ba67a..4e30b07 100644
--- a/ql/src/test/results/clientpositive/perf/spark/query54.q.out
+++ b/ql/src/test/results/clientpositive/perf/spark/query54.q.out
@@ -171,9 +171,9 @@ STAGE PLANS:
         Reducer 23 <- Reducer 22 (GROUP, 1)
         Reducer 25 <- Map 24 (GROUP, 2)
         Reducer 26 <- Reducer 25 (GROUP, 1)
-        Reducer 28 <- Map 24 (GROUP, 2)
+        Reducer 28 <- Map 27 (GROUP, 2)
         Reducer 3 <- Reducer 10 (PARTITION-LEVEL SORT, 772), Reducer 2 (PARTITION-LEVEL SORT, 772)
-        Reducer 30 <- Map 21 (GROUP, 2)
+        Reducer 30 <- Map 29 (GROUP, 2)
         Reducer 4 <- Reducer 23 (PARTITION-LEVEL SORT, 1), Reducer 26 (PARTITION-LEVEL SORT, 1), Reducer 28 (PARTITION-LEVEL SORT, 1), Reducer 3 (PARTITION-LEVEL SORT, 1), Reducer 30 (PARTITION-LEVEL SORT, 1)
         Reducer 5 <- Reducer 4 (GROUP, 1009)
         Reducer 6 <- Reducer 5 (GROUP, 1009)
@@ -348,14 +348,64 @@ STAGE PLANS:
                           Map-reduce partition columns: _col0 (type: int)
                           Statistics: Num rows: 18262 Data size: 20435178 Basic stats: COMPLETE Column stats: NONE
             Execution mode: vectorized
+        Map 27 
+            Map Operator Tree:
+                TableScan
+                  alias: date_dim
+                  filterExpr: ((d_year = 1999) and (d_moy = 3) and d_month_seq is not null) (type: boolean)
+                  Statistics: Num rows: 73049 Data size: 81741831 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: ((d_moy = 3) and (d_year = 1999) and d_month_seq is not null) (type: boolean)
+                    Statistics: Num rows: 18262 Data size: 20435178 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: (d_month_seq + 1) (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 18262 Data size: 20435178 Basic stats: COMPLETE Column stats: NONE
+                      Group By Operator
+                        keys: _col0 (type: int)
+                        minReductionHashAggr: 0.99
+                        mode: hash
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 18262 Data size: 20435178 Basic stats: COMPLETE Column stats: NONE
+                        Reduce Output Operator
+                          key expressions: _col0 (type: int)
+                          sort order: +
+                          Map-reduce partition columns: _col0 (type: int)
+                          Statistics: Num rows: 18262 Data size: 20435178 Basic stats: COMPLETE Column stats: NONE
+            Execution mode: vectorized
+        Map 29 
+            Map Operator Tree:
+                TableScan
+                  alias: date_dim
+                  filterExpr: ((d_year = 1999) and (d_moy = 3) and d_month_seq is not null) (type: boolean)
+                  Statistics: Num rows: 73049 Data size: 81741831 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: ((d_moy = 3) and (d_year = 1999) and d_month_seq is not null) (type: boolean)
+                    Statistics: Num rows: 18262 Data size: 20435178 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: (d_month_seq + 3) (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 18262 Data size: 20435178 Basic stats: COMPLETE Column stats: NONE
+                      Group By Operator
+                        keys: _col0 (type: int)
+                        minReductionHashAggr: 0.99
+                        mode: hash
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 18262 Data size: 20435178 Basic stats: COMPLETE Column stats: NONE
+                        Reduce Output Operator
+                          key expressions: _col0 (type: int)
+                          sort order: +
+                          Map-reduce partition columns: _col0 (type: int)
+                          Statistics: Num rows: 18262 Data size: 20435178 Basic stats: COMPLETE Column stats: NONE
+            Execution mode: vectorized
         Map 8 
             Map Operator Tree:
                 TableScan
                   alias: date_dim
-                  filterExpr: d_date_sk is not null (type: boolean)
+                  filterExpr: (d_date_sk is not null and d_month_seq is not null) (type: boolean)
                   Statistics: Num rows: 73049 Data size: 81741831 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: d_date_sk is not null (type: boolean)
+                    predicate: (d_date_sk is not null and d_month_seq is not null) (type: boolean)
                     Statistics: Num rows: 73049 Data size: 81741831 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: d_date_sk (type: int), d_month_seq (type: int)
diff --git a/ql/src/test/results/clientpositive/perf/spark/query6.q.out b/ql/src/test/results/clientpositive/perf/spark/query6.q.out
index 2573601..a828729 100644
--- a/ql/src/test/results/clientpositive/perf/spark/query6.q.out
+++ b/ql/src/test/results/clientpositive/perf/spark/query6.q.out
@@ -1,4 +1,4 @@
-Warning: Map Join MAPJOIN[85][bigTable=?] in task 'Stage-1:MAPRED' is a cross product
+Warning: Map Join MAPJOIN[86][bigTable=?] in task 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: explain
 select  a.ca_state state, count(*) cnt
  from customer_address a
@@ -261,10 +261,10 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: i
-                  filterExpr: (i_item_sk is not null and i_category is not null) (type: boolean)
+                  filterExpr: (i_item_sk is not null and i_category is not null and i_current_price is not null) (type: boolean)
                   Statistics: Num rows: 462000 Data size: 663560457 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: (i_category is not null and i_item_sk is not null) (type: boolean)
+                    predicate: (i_category is not null and i_current_price is not null and i_item_sk is not null) (type: boolean)
                     Statistics: Num rows: 462000 Data size: 663560457 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: i_item_sk (type: int), i_current_price (type: decimal(7,2)), i_category (type: string)
@@ -329,26 +329,29 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 231000 Data size: 331780228 Basic stats: COMPLETE Column stats: NONE
-                Select Operator
-                  expressions: _col0 (type: string), (1.2 * CAST( (_col1 / _col2) AS decimal(16,6))) (type: decimal(19,7))
-                  outputColumnNames: _col0, _col1
+                Filter Operator
+                  predicate: CAST( (_col1 / _col2) AS decimal(16,6)) is not null (type: boolean)
                   Statistics: Num rows: 231000 Data size: 331780228 Basic stats: COMPLETE Column stats: NONE
-                  Map Join Operator
-                    condition map:
-                         Inner Join 0 to 1
-                    keys:
-                      0 
-                      1 
+                  Select Operator
+                    expressions: _col0 (type: string), (1.2 * CAST( (_col1 / _col2) AS decimal(16,6))) (type: decimal(19,7))
                     outputColumnNames: _col0, _col1
-                    input vertices:
-                      1 Reducer 19
-                    Statistics: Num rows: 231000 Data size: 333859228 Basic stats: COMPLETE Column stats: NONE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: string)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: string)
+                    Statistics: Num rows: 231000 Data size: 331780228 Basic stats: COMPLETE Column stats: NONE
+                    Map Join Operator
+                      condition map:
+                           Inner Join 0 to 1
+                      keys:
+                        0 
+                        1 
+                      outputColumnNames: _col0, _col1
+                      input vertices:
+                        1 Reducer 19
                       Statistics: Num rows: 231000 Data size: 333859228 Basic stats: COMPLETE Column stats: NONE
-                      value expressions: _col1 (type: decimal(19,7))
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 231000 Data size: 333859228 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col1 (type: decimal(19,7))
         Reducer 16 
             Reduce Operator Tree:
               Join Operator
diff --git a/ql/src/test/results/clientpositive/perf/spark/query65.q.out b/ql/src/test/results/clientpositive/perf/spark/query65.q.out
index 242b730..5aaf3be 100644
--- a/ql/src/test/results/clientpositive/perf/spark/query65.q.out
+++ b/ql/src/test/results/clientpositive/perf/spark/query65.q.out
@@ -218,16 +218,19 @@ STAGE PLANS:
                     mode: complete
                     outputColumnNames: _col0, _col1, _col2
                     Statistics: Num rows: 158398803 Data size: 13973988377 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: _col0 (type: int), (0.1 * (_col1 / _col2)) (type: decimal(38,12))
-                      outputColumnNames: _col0, _col1
+                    Filter Operator
+                      predicate: (_col1 is not null and _col2 is not null) (type: boolean)
                       Statistics: Num rows: 158398803 Data size: 13973988377 Basic stats: COMPLETE Column stats: NONE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: int)
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
+                      Select Operator
+                        expressions: _col0 (type: int), (0.1 * (_col1 / _col2)) (type: decimal(38,12))
+                        outputColumnNames: _col0, _col1
                         Statistics: Num rows: 158398803 Data size: 13973988377 Basic stats: COMPLETE Column stats: NONE
-                        value expressions: _col1 (type: decimal(38,12))
+                        Reduce Output Operator
+                          key expressions: _col0 (type: int)
+                          sort order: +
+                          Map-reduce partition columns: _col0 (type: int)
+                          Statistics: Num rows: 158398803 Data size: 13973988377 Basic stats: COMPLETE Column stats: NONE
+                          value expressions: _col1 (type: decimal(38,12))
         Reducer 2 
             Reduce Operator Tree:
               Join Operator
@@ -260,12 +263,15 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 316797606 Data size: 27947976754 Basic stats: COMPLETE Column stats: NONE
-                Reduce Output Operator
-                  key expressions: _col0 (type: int)
-                  sort order: +
-                  Map-reduce partition columns: _col0 (type: int)
+                Filter Operator
+                  predicate: _col2 is not null (type: boolean)
                   Statistics: Num rows: 316797606 Data size: 27947976754 Basic stats: COMPLETE Column stats: NONE
-                  value expressions: _col1 (type: int), _col2 (type: decimal(17,2))
+                  Reduce Output Operator
+                    key expressions: _col0 (type: int)
+                    sort order: +
+                    Map-reduce partition columns: _col0 (type: int)
+                    Statistics: Num rows: 316797606 Data size: 27947976754 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col1 (type: int), _col2 (type: decimal(17,2))
         Reducer 4 
             Reduce Operator Tree:
               Join Operator
diff --git a/ql/src/test/results/clientpositive/perf/spark/query72.q.out b/ql/src/test/results/clientpositive/perf/spark/query72.q.out
index 66de773..d562173 100644
--- a/ql/src/test/results/clientpositive/perf/spark/query72.q.out
+++ b/ql/src/test/results/clientpositive/perf/spark/query72.q.out
@@ -173,10 +173,10 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: inventory
-                  filterExpr: (inv_item_sk is not null and inv_warehouse_sk is not null and inv_date_sk is not null) (type: boolean)
+                  filterExpr: (inv_item_sk is not null and inv_warehouse_sk is not null and inv_date_sk is not null and inv_quantity_on_hand is not null) (type: boolean)
                   Statistics: Num rows: 37584000 Data size: 593821104 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: (inv_date_sk is not null and inv_item_sk is not null and inv_warehouse_sk is not null) (type: boolean)
+                    predicate: (inv_date_sk is not null and inv_item_sk is not null and inv_quantity_on_hand is not null and inv_warehouse_sk is not null) (type: boolean)
                     Statistics: Num rows: 37584000 Data size: 593821104 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: inv_date_sk (type: int), inv_item_sk (type: int), inv_warehouse_sk (type: int), inv_quantity_on_hand (type: int)
@@ -205,10 +205,10 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: d1
-                  filterExpr: ((d_year = 2001) and d_date_sk is not null and d_week_seq is not null) (type: boolean)
+                  filterExpr: ((d_year = 2001) and d_date_sk is not null and d_week_seq is not null and UDFToDouble(d_date) is not null) (type: boolean)
                   Statistics: Num rows: 73049 Data size: 81741831 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: ((d_year = 2001) and d_date_sk is not null and d_week_seq is not null) (type: boolean)
+                    predicate: ((d_year = 2001) and UDFToDouble(d_date) is not null and d_date_sk is not null and d_week_seq is not null) (type: boolean)
                     Statistics: Num rows: 36524 Data size: 40870356 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: d_date_sk (type: int), d_week_seq (type: int), (UDFToDouble(d_date) + 5.0D) (type: double)
@@ -264,10 +264,10 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: d3
-                  filterExpr: d_date_sk is not null (type: boolean)
+                  filterExpr: (d_date_sk is not null and UDFToDouble(d_date) is not null) (type: boolean)
                   Statistics: Num rows: 73049 Data size: 81741831 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: d_date_sk is not null (type: boolean)
+                    predicate: (UDFToDouble(d_date) is not null and d_date_sk is not null) (type: boolean)
                     Statistics: Num rows: 73049 Data size: 81741831 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: d_date_sk (type: int), UDFToDouble(d_date) (type: double)
@@ -322,10 +322,10 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: catalog_sales
-                  filterExpr: (cs_item_sk is not null and cs_bill_cdemo_sk is not null and cs_bill_hdemo_sk is not null and cs_sold_date_sk is not null and cs_ship_date_sk is not null) (type: boolean)
+                  filterExpr: (cs_item_sk is not null and cs_bill_cdemo_sk is not null and cs_bill_hdemo_sk is not null and cs_sold_date_sk is not null and cs_ship_date_sk is not null and cs_quantity is not null) (type: boolean)
                   Statistics: Num rows: 287989836 Data size: 38999608952 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: (cs_bill_cdemo_sk is not null and cs_bill_hdemo_sk is not null and cs_item_sk is not null and cs_ship_date_sk is not null and cs_sold_date_sk is not null) (type: boolean)
+                    predicate: (cs_bill_cdemo_sk is not null and cs_bill_hdemo_sk is not null and cs_item_sk is not null and cs_quantity is not null and cs_ship_date_sk is not null and cs_sold_date_sk is not null) (type: boolean)
                     Statistics: Num rows: 287989836 Data size: 38999608952 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: cs_sold_date_sk (type: int), cs_ship_date_sk (type: int), cs_bill_cdemo_sk (type: int), cs_bill_hdemo_sk (type: int), cs_item_sk (type: int), cs_promo_sk (type: int), cs_order_number (type: int), cs_quantity (type: int)
diff --git a/ql/src/test/results/clientpositive/perf/spark/query81.q.out b/ql/src/test/results/clientpositive/perf/spark/query81.q.out
index 03ab97e..bcd1a5e 100644
--- a/ql/src/test/results/clientpositive/perf/spark/query81.q.out
+++ b/ql/src/test/results/clientpositive/perf/spark/query81.q.out
@@ -327,16 +327,19 @@ STAGE PLANS:
                     mode: complete
                     outputColumnNames: _col0, _col1, _col2
                     Statistics: Num rows: 11000000 Data size: 11163678945 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: ((_col1 / _col2) * 1.2) (type: decimal(38,11)), _col0 (type: string)
-                      outputColumnNames: _col0, _col1
+                    Filter Operator
+                      predicate: (_col1 is not null and _col2 is not null) (type: boolean)
                       Statistics: Num rows: 11000000 Data size: 11163678945 Basic stats: COMPLETE Column stats: NONE
-                      Reduce Output Operator
-                        key expressions: _col1 (type: string)
-                        sort order: +
-                        Map-reduce partition columns: _col1 (type: string)
+                      Select Operator
+                        expressions: ((_col1 / _col2) * 1.2) (type: decimal(38,11)), _col0 (type: string)
+                        outputColumnNames: _col0, _col1
                         Statistics: Num rows: 11000000 Data size: 11163678945 Basic stats: COMPLETE Column stats: NONE
-                        value expressions: _col0 (type: decimal(38,11))
+                        Reduce Output Operator
+                          key expressions: _col1 (type: string)
+                          sort order: +
+                          Map-reduce partition columns: _col1 (type: string)
+                          Statistics: Num rows: 11000000 Data size: 11163678945 Basic stats: COMPLETE Column stats: NONE
+                          value expressions: _col0 (type: decimal(38,11))
         Reducer 2 
             Reduce Operator Tree:
               Join Operator
@@ -445,12 +448,15 @@ STAGE PLANS:
                   expressions: _col1 (type: int), _col0 (type: string), _col2 (type: decimal(17,2))
                   outputColumnNames: _col0, _col1, _col2
                   Statistics: Num rows: 22000000 Data size: 22327357890 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col1 (type: string)
-                    sort order: +
-                    Map-reduce partition columns: _col1 (type: string)
+                  Filter Operator
+                    predicate: _col2 is not null (type: boolean)
                     Statistics: Num rows: 22000000 Data size: 22327357890 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col0 (type: int), _col2 (type: decimal(17,2))
+                    Reduce Output Operator
+                      key expressions: _col1 (type: string)
+                      sort order: +
+                      Map-reduce partition columns: _col1 (type: string)
+                      Statistics: Num rows: 22000000 Data size: 22327357890 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col0 (type: int), _col2 (type: decimal(17,2))
 
   Stage: Stage-0
     Fetch Operator
diff --git a/ql/src/test/results/clientpositive/perf/spark/query92.q.out b/ql/src/test/results/clientpositive/perf/spark/query92.q.out
index 535775a..b620a7b 100644
--- a/ql/src/test/results/clientpositive/perf/spark/query92.q.out
+++ b/ql/src/test/results/clientpositive/perf/spark/query92.q.out
@@ -124,20 +124,19 @@ STAGE PLANS:
   Stage: Stage-1
     Spark
       Edges:
-        Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 261), Reducer 7 (PARTITION-LEVEL SORT, 261)
+        Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 255), Map 5 (PARTITION-LEVEL SORT, 255), Reducer 7 (PARTITION-LEVEL SORT, 255)
         Reducer 3 <- Reducer 2 (GROUP, 1)
-        Reducer 6 <- Map 5 (GROUP, 169)
-        Reducer 7 <- Map 9 (PARTITION-LEVEL SORT, 87), Reducer 6 (PARTITION-LEVEL SORT, 87)
+        Reducer 7 <- Map 6 (GROUP, 169)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: web_sales
-                  filterExpr: (ws_item_sk is not null and ws_sold_date_sk is not null) (type: boolean)
+                  filterExpr: (ws_item_sk is not null and ws_sold_date_sk is not null and ws_ext_discount_amt is not null) (type: boolean)
                   Statistics: Num rows: 144002668 Data size: 19580198212 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: (ws_item_sk is not null and ws_sold_date_sk is not null) (type: boolean)
+                    predicate: (ws_ext_discount_amt is not null and ws_item_sk is not null and ws_sold_date_sk is not null) (type: boolean)
                     Statistics: Num rows: 144002668 Data size: 19580198212 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: ws_sold_date_sk (type: int), ws_item_sk (type: int), ws_ext_discount_amt (type: decimal(7,2))
@@ -165,6 +164,25 @@ STAGE PLANS:
         Map 5 
             Map Operator Tree:
                 TableScan
+                  alias: item
+                  filterExpr: ((i_manufact_id = 269) and i_item_sk is not null) (type: boolean)
+                  Statistics: Num rows: 462000 Data size: 663560457 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: ((i_manufact_id = 269) and i_item_sk is not null) (type: boolean)
+                    Statistics: Num rows: 231000 Data size: 331780228 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: i_item_sk (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 231000 Data size: 331780228 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 231000 Data size: 331780228 Basic stats: COMPLETE Column stats: NONE
+            Execution mode: vectorized
+        Map 6 
+            Map Operator Tree:
+                TableScan
                   alias: web_sales
                   filterExpr: (ws_item_sk is not null and ws_sold_date_sk is not null) (type: boolean)
                   Statistics: Num rows: 144002668 Data size: 19580198212 Basic stats: COMPLETE Column stats: NONE
@@ -201,42 +219,25 @@ STAGE PLANS:
             Execution mode: vectorized
             Local Work:
               Map Reduce Local Work
-        Map 9 
-            Map Operator Tree:
-                TableScan
-                  alias: item
-                  filterExpr: ((i_manufact_id = 269) and i_item_sk is not null) (type: boolean)
-                  Statistics: Num rows: 462000 Data size: 663560457 Basic stats: COMPLETE Column stats: NONE
-                  Filter Operator
-                    predicate: ((i_manufact_id = 269) and i_item_sk is not null) (type: boolean)
-                    Statistics: Num rows: 231000 Data size: 331780228 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: i_item_sk (type: int)
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 231000 Data size: 331780228 Basic stats: COMPLETE Column stats: NONE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: int)
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 231000 Data size: 331780228 Basic stats: COMPLETE Column stats: NONE
-            Execution mode: vectorized
         Reducer 2 
             Reduce Operator Tree:
               Join Operator
                 condition map:
                      Inner Join 0 to 1
+                     Inner Join 1 to 2
                 keys:
                   0 _col1 (type: int)
-                  1 _col2 (type: int)
-                outputColumnNames: _col2, _col5
-                Statistics: Num rows: 174243235 Data size: 23692040863 Basic stats: COMPLETE Column stats: NONE
+                  1 _col0 (type: int)
+                  2 _col0 (type: int)
+                outputColumnNames: _col2, _col6
+                Statistics: Num rows: 348486471 Data size: 47384081727 Basic stats: COMPLETE Column stats: NONE
                 Filter Operator
-                  predicate: (_col2 > _col5) (type: boolean)
-                  Statistics: Num rows: 58081078 Data size: 7897346909 Basic stats: COMPLETE Column stats: NONE
+                  predicate: (_col2 > _col6) (type: boolean)
+                  Statistics: Num rows: 116162157 Data size: 15794693909 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: _col2 (type: decimal(7,2))
                     outputColumnNames: _col2
-                    Statistics: Num rows: 58081078 Data size: 7897346909 Basic stats: COMPLETE Column stats: NONE
+                    Statistics: Num rows: 116162157 Data size: 15794693909 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
                       aggregations: sum(_col2)
                       minReductionHashAggr: 0.99
@@ -262,7 +263,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 7 
             Execution mode: vectorized
             Reduce Operator Tree:
               Group By Operator
@@ -271,32 +272,19 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 79201469 Data size: 10769109250 Basic stats: COMPLETE Column stats: NONE
-                Select Operator
-                  expressions: _col0 (type: int), CAST( (1.3 * (_col1 / _col2)) AS decimal(14,7)) (type: decimal(14,7))
-                  outputColumnNames: _col0, _col1
+                Filter Operator
+                  predicate: CAST( (1.3 * (_col1 / _col2)) AS decimal(14,7)) is not null (type: boolean)
                   Statistics: Num rows: 79201469 Data size: 10769109250 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: int)
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: int)
+                  Select Operator
+                    expressions: _col0 (type: int), CAST( (1.3 * (_col1 / _col2)) AS decimal(14,7)) (type: decimal(14,7))
+                    outputColumnNames: _col0, _col1
                     Statistics: Num rows: 79201469 Data size: 10769109250 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col1 (type: decimal(14,7))
-        Reducer 7 
-            Reduce Operator Tree:
-              Join Operator
-                condition map:
-                     Inner Join 0 to 1
-                keys:
-                  0 _col0 (type: int)
-                  1 _col0 (type: int)
-                outputColumnNames: _col1, _col2
-                Statistics: Num rows: 87121617 Data size: 11846020431 Basic stats: COMPLETE Column stats: NONE
-                Reduce Output Operator
-                  key expressions: _col2 (type: int)
-                  sort order: +
-                  Map-reduce partition columns: _col2 (type: int)
-                  Statistics: Num rows: 87121617 Data size: 11846020431 Basic stats: COMPLETE Column stats: NONE
-                  value expressions: _col1 (type: decimal(14,7))
+                    Reduce Output Operator
+                      key expressions: _col0 (type: int)
+                      sort order: +
+                      Map-reduce partition columns: _col0 (type: int)
+                      Statistics: Num rows: 79201469 Data size: 10769109250 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col1 (type: decimal(14,7))
 
   Stage: Stage-0
     Fetch Operator
diff --git a/ql/src/test/results/clientpositive/perf/tez/cbo_ext_query1.q.out b/ql/src/test/results/clientpositive/perf/tez/cbo_ext_query1.q.out
index 1d22045..f7a2a66 100644
--- a/ql/src/test/results/clientpositive/perf/tez/cbo_ext_query1.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/cbo_ext_query1.q.out
@@ -59,35 +59,37 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 CBO PLAN:
 HiveSortLimit(sort0=[$0], dir0=[ASC], fetch=[100]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
   HiveProject(c_customer_id=[$1]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-    HiveJoin(condition=[AND(=($3, $7), >($4, $6))], joinType=[inner], algorithm=[none], cost=[not available]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-      HiveJoin(condition=[=($2, $0)], joinType=[inner], algorithm=[none], cost=[not available]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+    HiveJoin(condition=[AND(=($4, $7), >($5, $6))], joinType=[inner], algorithm=[none], cost=[not available]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+      HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
         HiveProject(c_customer_sk=[$0], c_customer_id=[$1]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
           HiveFilter(condition=[IS NOT NULL($0)]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
             HiveTableScan(table=[[default, customer]], table:alias=[customer]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-        HiveJoin(condition=[=($3, $1)], joinType=[inner], algorithm=[none], cost=[not available]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-          HiveProject(sr_customer_sk=[$0], sr_store_sk=[$1], $f2=[$2]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-            HiveAggregate(group=[{1, 2}], agg#0=[sum($3)]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-              HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-                HiveProject(sr_returned_date_sk=[$0], sr_customer_sk=[$3], sr_store_sk=[$7], sr_fee=[$14]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-                  HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7), IS NOT NULL($3))]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-                    HiveTableScan(table=[[default, store_returns]], table:alias=[store_returns]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-                HiveProject(d_date_sk=[$0]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-                  HiveFilter(condition=[AND(=($6, 2000), IS NOT NULL($0))]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-                    HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+        HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
           HiveProject(s_store_sk=[$0]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
             HiveFilter(condition=[AND(=($24, _UTF-16LE'NM'), IS NOT NULL($0))]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
               HiveTableScan(table=[[default, store]], table:alias=[store]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-      HiveProject(_o__c0=[*(/($1, $2), 1.2:DECIMAL(2, 1))], ctr_store_sk=[$0]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-        HiveAggregate(group=[{1}], agg#0=[sum($2)], agg#1=[count($2)]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
           HiveProject(sr_customer_sk=[$0], sr_store_sk=[$1], $f2=[$2]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-            HiveAggregate(group=[{1, 2}], agg#0=[sum($3)]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-              HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-                HiveProject(sr_returned_date_sk=[$0], sr_customer_sk=[$3], sr_store_sk=[$7], sr_fee=[$14]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-                  HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7))]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-                    HiveTableScan(table=[[default, store_returns]], table:alias=[store_returns]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-                HiveProject(d_date_sk=[$0]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-                  HiveFilter(condition=[AND(=($6, 2000), IS NOT NULL($0))]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-                    HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+            HiveFilter(condition=[IS NOT NULL($2)]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+              HiveAggregate(group=[{1, 2}], agg#0=[sum($3)]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+                HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+                  HiveProject(sr_returned_date_sk=[$0], sr_customer_sk=[$3], sr_store_sk=[$7], sr_fee=[$14]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+                    HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7), IS NOT NULL($3))]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+                      HiveTableScan(table=[[default, store_returns]], table:alias=[store_returns]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+                  HiveProject(d_date_sk=[$0]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+                    HiveFilter(condition=[AND(=($6, 2000), IS NOT NULL($0))]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+                      HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+      HiveProject(_o__c0=[*(/($1, $2), 1.2:DECIMAL(2, 1))], ctr_store_sk=[$0]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+        HiveFilter(condition=[AND(IS NOT NULL($1), IS NOT NULL($2))]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+          HiveAggregate(group=[{1}], agg#0=[sum($2)], agg#1=[count($2)]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+            HiveProject(sr_customer_sk=[$0], sr_store_sk=[$1], $f2=[$2]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+              HiveAggregate(group=[{1, 2}], agg#0=[sum($3)]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+                HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+                  HiveProject(sr_returned_date_sk=[$0], sr_customer_sk=[$3], sr_store_sk=[$7], sr_fee=[$14]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+                    HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7))]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+                      HiveTableScan(table=[[default, store_returns]], table:alias=[store_returns]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+                  HiveProject(d_date_sk=[$0]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+                    HiveFilter(condition=[AND(=($6, 2000), IS NOT NULL($0))]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+                      HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
 
 PREHOOK: query: explain cbo joincost
 with customer_total_return as
@@ -150,33 +152,35 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 CBO PLAN:
 HiveSortLimit(sort0=[$0], dir0=[ASC], fetch=[100]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
   HiveProject(c_customer_id=[$1]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-    HiveJoin(condition=[AND(=($3, $7), >($4, $6))], joinType=[inner], algorithm=[none], cost=[{415687.382770037 rows, 0.0 cpu, 0.0 io}]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-      HiveJoin(condition=[=($2, $0)], joinType=[inner], algorithm=[none], cost=[{8.00093932086143E7 rows, 0.0 cpu, 0.0 io}]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+    HiveJoin(condition=[AND(=($4, $7), >($5, $6))], joinType=[inner], algorithm=[none], cost=[{374952.69636986067 rows, 0.0 cpu, 0.0 io}]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+      HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[{8.000928883962971E7 rows, 0.0 cpu, 0.0 io}]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
         HiveProject(c_customer_sk=[$0], c_customer_id=[$1]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
           HiveFilter(condition=[IS NOT NULL($0)]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
             HiveTableScan(table=[[default, customer]], table:alias=[customer]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-        HiveJoin(condition=[=($3, $1)], joinType=[inner], algorithm=[none], cost=[{460301.9976112889 rows, 0.0 cpu, 0.0 io}]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-          HiveProject(sr_customer_sk=[$0], sr_store_sk=[$1], $f2=[$2]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-            HiveAggregate(group=[{1, 2}], agg#0=[sum($3)]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-              HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[{5.175767820386722E7 rows, 0.0 cpu, 0.0 io}]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-                HiveProject(sr_returned_date_sk=[$0], sr_customer_sk=[$3], sr_store_sk=[$7], sr_fee=[$14]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-                  HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7), IS NOT NULL($3))]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-                    HiveTableScan(table=[[default, store_returns]], table:alias=[store_returns]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-                HiveProject(d_date_sk=[$0]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-                  HiveFilter(condition=[AND(=($6, 2000), IS NOT NULL($0))]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-                    HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+        HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[{455187.9173657213 rows, 0.0 cpu, 0.0 io}]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
           HiveProject(s_store_sk=[$0]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
             HiveFilter(condition=[AND(=($24, _UTF-16LE'NM'), IS NOT NULL($0))]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
               HiveTableScan(table=[[default, store]], table:alias=[store]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-      HiveProject(_o__c0=[*(/($1, $2), 1.2:DECIMAL(2, 1))], ctr_store_sk=[$0]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-        HiveAggregate(group=[{1}], agg#0=[sum($2)], agg#1=[count($2)]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
           HiveProject(sr_customer_sk=[$0], sr_store_sk=[$1], $f2=[$2]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-            HiveAggregate(group=[{1, 2}], agg#0=[sum($3)]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-              HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[{5.3635511784936875E7 rows, 0.0 cpu, 0.0 io}]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-                HiveProject(sr_returned_date_sk=[$0], sr_customer_sk=[$3], sr_store_sk=[$7], sr_fee=[$14]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-                  HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7))]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-                    HiveTableScan(table=[[default, store_returns]], table:alias=[store_returns]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-                HiveProject(d_date_sk=[$0]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-                  HiveFilter(condition=[AND(=($6, 2000), IS NOT NULL($0))]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-                    HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+            HiveFilter(condition=[IS NOT NULL($2)]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+              HiveAggregate(group=[{1, 2}], agg#0=[sum($3)]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+                HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[{5.175767820386722E7 rows, 0.0 cpu, 0.0 io}]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+                  HiveProject(sr_returned_date_sk=[$0], sr_customer_sk=[$3], sr_store_sk=[$7], sr_fee=[$14]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+                    HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7), IS NOT NULL($3))]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+                      HiveTableScan(table=[[default, store_returns]], table:alias=[store_returns]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+                  HiveProject(d_date_sk=[$0]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+                    HiveFilter(condition=[AND(=($6, 2000), IS NOT NULL($0))]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+                      HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+      HiveProject(_o__c0=[*(/($1, $2), 1.2:DECIMAL(2, 1))], ctr_store_sk=[$0]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+        HiveFilter(condition=[AND(IS NOT NULL($1), IS NOT NULL($2))]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+          HiveAggregate(group=[{1}], agg#0=[sum($2)], agg#1=[count($2)]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+            HiveProject(sr_customer_sk=[$0], sr_store_sk=[$1], $f2=[$2]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+              HiveAggregate(group=[{1, 2}], agg#0=[sum($3)]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+                HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[{5.3635511784936875E7 rows, 0.0 cpu, 0.0 io}]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+                  HiveProject(sr_returned_date_sk=[$0], sr_customer_sk=[$3], sr_store_sk=[$7], sr_fee=[$14]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+                    HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7))]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+                      HiveTableScan(table=[[default, store_returns]], table:alias=[store_returns]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+                  HiveProject(d_date_sk=[$0]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+                    HiveFilter(condition=[AND(=($6, 2000), IS NOT NULL($0))]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+                      HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/cbo_query1.q.out b/ql/src/test/results/clientpositive/perf/tez/cbo_query1.q.out
index d59f9f5..a52ff24 100644
--- a/ql/src/test/results/clientpositive/perf/tez/cbo_query1.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/cbo_query1.q.out
@@ -59,33 +59,35 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 CBO PLAN:
 HiveSortLimit(sort0=[$0], dir0=[ASC], fetch=[100])
   HiveProject(c_customer_id=[$1])
-    HiveJoin(condition=[AND(=($3, $7), >($4, $6))], joinType=[inner], algorithm=[none], cost=[not available])
-      HiveJoin(condition=[=($2, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+    HiveJoin(condition=[AND(=($4, $7), >($5, $6))], joinType=[inner], algorithm=[none], cost=[not available])
+      HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
         HiveProject(c_customer_sk=[$0], c_customer_id=[$1])
           HiveFilter(condition=[IS NOT NULL($0)])
             HiveTableScan(table=[[default, customer]], table:alias=[customer])
-        HiveJoin(condition=[=($3, $1)], joinType=[inner], algorithm=[none], cost=[not available])
-          HiveProject(sr_customer_sk=[$0], sr_store_sk=[$1], $f2=[$2])
-            HiveAggregate(group=[{1, 2}], agg#0=[sum($3)])
-              HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available])
-                HiveProject(sr_returned_date_sk=[$0], sr_customer_sk=[$3], sr_store_sk=[$7], sr_fee=[$14])
-                  HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7), IS NOT NULL($3))])
-                    HiveTableScan(table=[[default, store_returns]], table:alias=[store_returns])
-                HiveProject(d_date_sk=[$0])
-                  HiveFilter(condition=[AND(=($6, 2000), IS NOT NULL($0))])
-                    HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+        HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
           HiveProject(s_store_sk=[$0])
             HiveFilter(condition=[AND(=($24, _UTF-16LE'NM'), IS NOT NULL($0))])
               HiveTableScan(table=[[default, store]], table:alias=[store])
-      HiveProject(_o__c0=[*(/($1, $2), 1.2:DECIMAL(2, 1))], ctr_store_sk=[$0])
-        HiveAggregate(group=[{1}], agg#0=[sum($2)], agg#1=[count($2)])
           HiveProject(sr_customer_sk=[$0], sr_store_sk=[$1], $f2=[$2])
-            HiveAggregate(group=[{1, 2}], agg#0=[sum($3)])
-              HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available])
-                HiveProject(sr_returned_date_sk=[$0], sr_customer_sk=[$3], sr_store_sk=[$7], sr_fee=[$14])
-                  HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7))])
-                    HiveTableScan(table=[[default, store_returns]], table:alias=[store_returns])
-                HiveProject(d_date_sk=[$0])
-                  HiveFilter(condition=[AND(=($6, 2000), IS NOT NULL($0))])
-                    HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+            HiveFilter(condition=[IS NOT NULL($2)])
+              HiveAggregate(group=[{1, 2}], agg#0=[sum($3)])
+                HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available])
+                  HiveProject(sr_returned_date_sk=[$0], sr_customer_sk=[$3], sr_store_sk=[$7], sr_fee=[$14])
+                    HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7), IS NOT NULL($3))])
+                      HiveTableScan(table=[[default, store_returns]], table:alias=[store_returns])
+                  HiveProject(d_date_sk=[$0])
+                    HiveFilter(condition=[AND(=($6, 2000), IS NOT NULL($0))])
+                      HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+      HiveProject(_o__c0=[*(/($1, $2), 1.2:DECIMAL(2, 1))], ctr_store_sk=[$0])
+        HiveFilter(condition=[AND(IS NOT NULL($1), IS NOT NULL($2))])
+          HiveAggregate(group=[{1}], agg#0=[sum($2)], agg#1=[count($2)])
+            HiveProject(sr_customer_sk=[$0], sr_store_sk=[$1], $f2=[$2])
+              HiveAggregate(group=[{1, 2}], agg#0=[sum($3)])
+                HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available])
+                  HiveProject(sr_returned_date_sk=[$0], sr_customer_sk=[$3], sr_store_sk=[$7], sr_fee=[$14])
+                    HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7))])
+                      HiveTableScan(table=[[default, store_returns]], table:alias=[store_returns])
+                  HiveProject(d_date_sk=[$0])
+                    HiveFilter(condition=[AND(=($6, 2000), IS NOT NULL($0))])
+                      HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/cbo_query14.q.out b/ql/src/test/results/clientpositive/perf/tez/cbo_query14.q.out
index 91032b7..e783983 100644
--- a/ql/src/test/results/clientpositive/perf/tez/cbo_query14.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/cbo_query14.q.out
@@ -1,6 +1,6 @@
-Warning: Shuffle Join MERGEJOIN[1164][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 6' is a cross product
-Warning: Shuffle Join MERGEJOIN[1171][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 16' is a cross product
-Warning: Shuffle Join MERGEJOIN[1178][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 22' is a cross product
+Warning: Shuffle Join MERGEJOIN[1173][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 6' is a cross product
+Warning: Shuffle Join MERGEJOIN[1180][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 16' is a cross product
+Warning: Shuffle Join MERGEJOIN[1187][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 22' is a cross product
 PREHOOK: query: explain cbo
 with  cross_items as
  (select i_item_sk ss_item_sk
@@ -228,283 +228,292 @@ HiveSortLimit(sort0=[$0], sort1=[$1], sort2=[$2], sort3=[$3], dir0=[ASC], dir1=[
           HiveProject(channel=[_UTF-16LE'store':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"], i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], sales=[$3], number_sales=[$4])
             HiveJoin(condition=[>($3, $5)], joinType=[inner], algorithm=[none], cost=[not available])
               HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3], $f4=[$4])
-                HiveAggregate(group=[{0, 1, 2}], agg#0=[sum($3)], agg#1=[count()])
-                  HiveProject(i_brand_id=[$1], i_class_id=[$2], i_category_id=[$3], $f3=[*(CAST($6):DECIMAL(10, 0), $7)])
-                    HiveSemiJoin(condition=[=($5, $9)], joinType=[inner])
-                      HiveJoin(condition=[=($5, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                        HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
-                          HiveFilter(condition=[IS NOT NULL($0)])
-                            HiveTableScan(table=[[default, item]], table:alias=[item])
-                        HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available])
-                          HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2], ss_quantity=[$10], ss_list_price=[$12])
-                            HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($0))])
-                              HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                          HiveProject(d_date_sk=[$0])
-                            HiveFilter(condition=[AND(=($6, 2000), =($8, 11), IS NOT NULL($0))])
-                              HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-                      HiveProject(i_item_sk=[$0])
-                        HiveJoin(condition=[AND(=($1, $4), =($2, $5), =($3, $6))], joinType=[inner], algorithm=[none], cost=[not available])
+                HiveFilter(condition=[IS NOT NULL($3)])
+                  HiveAggregate(group=[{0, 1, 2}], agg#0=[sum($3)], agg#1=[count()])
+                    HiveProject(i_brand_id=[$1], i_class_id=[$2], i_category_id=[$3], $f3=[*(CAST($6):DECIMAL(10, 0), $7)])
+                      HiveSemiJoin(condition=[=($5, $9)], joinType=[inner])
+                        HiveJoin(condition=[=($5, $0)], joinType=[inner], algorithm=[none], cost=[not available])
                           HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
-                            HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11), IS NOT NULL($0))])
+                            HiveFilter(condition=[IS NOT NULL($0)])
                               HiveTableScan(table=[[default, item]], table:alias=[item])
-                          HiveProject($f0=[$0], $f1=[$1], $f2=[$2])
-                            HiveFilter(condition=[=($3, 3)])
-                              HiveAggregate(group=[{0, 1, 2}], agg#0=[count($3)])
-                                HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3])
-                                  HiveUnion(all=[true])
-                                    HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3])
-                                      HiveAggregate(group=[{4, 5, 6}], agg#0=[count()])
-                                        HiveJoin(condition=[=($1, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                                          HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-                                            HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2])
-                                              HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($0))])
-                                                HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                                            HiveProject(d_date_sk=[$0])
-                                              HiveFilter(condition=[AND(BETWEEN(false, $6, 1999, 2001), IS NOT NULL($0))])
-                                                HiveTableScan(table=[[default, date_dim]], table:alias=[d1])
-                                          HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
-                                            HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11))])
-                                              HiveTableScan(table=[[default, item]], table:alias=[iss])
-                                    HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3])
-                                      HiveAggregate(group=[{4, 5, 6}], agg#0=[count()])
-                                        HiveJoin(condition=[=($1, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                                          HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-                                            HiveProject(cs_sold_date_sk=[$0], cs_item_sk=[$15])
-                                              HiveFilter(condition=[AND(IS NOT NULL($15), IS NOT NULL($0))])
-                                                HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
-                                            HiveProject(d_date_sk=[$0])
-                                              HiveFilter(condition=[AND(BETWEEN(false, $6, 1999, 2001), IS NOT NULL($0))])
-                                                HiveTableScan(table=[[default, date_dim]], table:alias=[d2])
-                                          HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
-                                            HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11))])
-                                              HiveTableScan(table=[[default, item]], table:alias=[ics])
-                                    HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3])
-                                      HiveAggregate(group=[{4, 5, 6}], agg#0=[count()])
-                                        HiveJoin(condition=[=($1, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                                          HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-                                            HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$3])
-                                              HiveFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($0))])
-                                                HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
-                                            HiveProject(d_date_sk=[$0])
-                                              HiveFilter(condition=[AND(BETWEEN(false, $6, 1999, 2001), IS NOT NULL($0))])
-                                                HiveTableScan(table=[[default, date_dim]], table:alias=[d3])
-                                          HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
-                                            HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11))])
-                                              HiveTableScan(table=[[default, item]], table:alias=[iws])
+                          HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available])
+                            HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2], ss_quantity=[$10], ss_list_price=[$12])
+                              HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($0))])
+                                HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                            HiveProject(d_date_sk=[$0])
+                              HiveFilter(condition=[AND(=($6, 2000), =($8, 11), IS NOT NULL($0))])
+                                HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+                        HiveProject(i_item_sk=[$0])
+                          HiveJoin(condition=[AND(=($1, $4), =($2, $5), =($3, $6))], joinType=[inner], algorithm=[none], cost=[not available])
+                            HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
+                              HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11), IS NOT NULL($0))])
+                                HiveTableScan(table=[[default, item]], table:alias=[item])
+                            HiveProject($f0=[$0], $f1=[$1], $f2=[$2])
+                              HiveFilter(condition=[=($3, 3)])
+                                HiveAggregate(group=[{0, 1, 2}], agg#0=[count($3)])
+                                  HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3])
+                                    HiveUnion(all=[true])
+                                      HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3])
+                                        HiveAggregate(group=[{4, 5, 6}], agg#0=[count()])
+                                          HiveJoin(condition=[=($1, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                                            HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+                                              HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2])
+                                                HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($0))])
+                                                  HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                                              HiveProject(d_date_sk=[$0])
+                                                HiveFilter(condition=[AND(BETWEEN(false, $6, 1999, 2001), IS NOT NULL($0))])
+                                                  HiveTableScan(table=[[default, date_dim]], table:alias=[d1])
+                                            HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
+                                              HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11))])
+                                                HiveTableScan(table=[[default, item]], table:alias=[iss])
+                                      HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3])
+                                        HiveAggregate(group=[{4, 5, 6}], agg#0=[count()])
+                                          HiveJoin(condition=[=($1, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                                            HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+                                              HiveProject(cs_sold_date_sk=[$0], cs_item_sk=[$15])
+                                                HiveFilter(condition=[AND(IS NOT NULL($15), IS NOT NULL($0))])
+                                                  HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
+                                              HiveProject(d_date_sk=[$0])
+                                                HiveFilter(condition=[AND(BETWEEN(false, $6, 1999, 2001), IS NOT NULL($0))])
+                                                  HiveTableScan(table=[[default, date_dim]], table:alias=[d2])
+                                            HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
+                                              HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11))])
+                                                HiveTableScan(table=[[default, item]], table:alias=[ics])
+                                      HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3])
+                                        HiveAggregate(group=[{4, 5, 6}], agg#0=[count()])
+                                          HiveJoin(condition=[=($1, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                                            HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+                                              HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$3])
+                                                HiveFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($0))])
+                                                  HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
+                                              HiveProject(d_date_sk=[$0])
+                                                HiveFilter(condition=[AND(BETWEEN(false, $6, 1999, 2001), IS NOT NULL($0))])
+                                                  HiveTableScan(table=[[default, date_dim]], table:alias=[d3])
+                                            HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
+                                              HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11))])
+                                                HiveTableScan(table=[[default, item]], table:alias=[iws])
               HiveProject($f0=[/($0, $1)])
-                HiveAggregate(group=[{}], agg#0=[sum($0)], agg#1=[count($0)])
-                  HiveProject($f0=[*(CAST($0):DECIMAL(10, 0), $1)])
-                    HiveUnion(all=[true])
-                      HiveProject(quantity=[$1], list_price=[$2])
-                        HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                          HiveProject(ss_sold_date_sk=[$0], ss_quantity=[$10], ss_list_price=[$12])
-                            HiveFilter(condition=[IS NOT NULL($0)])
-                              HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                          HiveProject(d_date_sk=[$0])
-                            HiveFilter(condition=[AND(BETWEEN(false, $6, 1999, 2001), IS NOT NULL($0))])
-                              HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-                      HiveProject(quantity=[$1], list_price=[$2])
-                        HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                          HiveProject(cs_sold_date_sk=[$0], cs_quantity=[$18], cs_list_price=[$20])
-                            HiveFilter(condition=[IS NOT NULL($0)])
-                              HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
-                          HiveProject(d_date_sk=[$0])
-                            HiveFilter(condition=[AND(BETWEEN(false, $6, 1998, 2000), IS NOT NULL($0))])
-                              HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-                      HiveProject(quantity=[$1], list_price=[$2])
-                        HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                          HiveProject(ws_sold_date_sk=[$0], ws_quantity=[$18], ws_list_price=[$20])
-                            HiveFilter(condition=[IS NOT NULL($0)])
-                              HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
-                          HiveProject(d_date_sk=[$0])
-                            HiveFilter(condition=[AND(BETWEEN(false, $6, 1998, 2000), IS NOT NULL($0))])
-                              HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+                HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($1))])
+                  HiveProject($f0=[$0], $f1=[$1])
+                    HiveAggregate(group=[{}], agg#0=[sum($0)], agg#1=[count($0)])
+                      HiveProject($f0=[*(CAST($0):DECIMAL(10, 0), $1)])
+                        HiveUnion(all=[true])
+                          HiveProject(quantity=[$1], list_price=[$2])
+                            HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                              HiveProject(ss_sold_date_sk=[$0], ss_quantity=[$10], ss_list_price=[$12])
+                                HiveFilter(condition=[IS NOT NULL($0)])
+                                  HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                              HiveProject(d_date_sk=[$0])
+                                HiveFilter(condition=[AND(BETWEEN(false, $6, 1999, 2001), IS NOT NULL($0))])
+                                  HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+                          HiveProject(quantity=[$1], list_price=[$2])
+                            HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                              HiveProject(cs_sold_date_sk=[$0], cs_quantity=[$18], cs_list_price=[$20])
+                                HiveFilter(condition=[IS NOT NULL($0)])
+                                  HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
+                              HiveProject(d_date_sk=[$0])
+                                HiveFilter(condition=[AND(BETWEEN(false, $6, 1998, 2000), IS NOT NULL($0))])
+                                  HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+                          HiveProject(quantity=[$1], list_price=[$2])
+                            HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                              HiveProject(ws_sold_date_sk=[$0], ws_quantity=[$18], ws_list_price=[$20])
+                                HiveFilter(condition=[IS NOT NULL($0)])
+                                  HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
+                              HiveProject(d_date_sk=[$0])
+                                HiveFilter(condition=[AND(BETWEEN(false, $6, 1998, 2000), IS NOT NULL($0))])
+                                  HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
           HiveProject(channel=[_UTF-16LE'catalog':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"], i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], sales=[$3], number_sales=[$4])
             HiveJoin(condition=[>($3, $5)], joinType=[inner], algorithm=[none], cost=[not available])
               HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3], $f4=[$4])
-                HiveAggregate(group=[{0, 1, 2}], agg#0=[sum($3)], agg#1=[count()])
-                  HiveProject(i_brand_id=[$1], i_class_id=[$2], i_category_id=[$3], $f3=[*(CAST($6):DECIMAL(10, 0), $7)])
-                    HiveSemiJoin(condition=[=($5, $9)], joinType=[inner])
-                      HiveJoin(condition=[=($5, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                        HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
-                          HiveFilter(condition=[IS NOT NULL($0)])
-                            HiveTableScan(table=[[default, item]], table:alias=[item])
-                        HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available])
-                          HiveProject(cs_sold_date_sk=[$0], cs_item_sk=[$15], cs_quantity=[$18], cs_list_price=[$20])
-                            HiveFilter(condition=[AND(IS NOT NULL($15), IS NOT NULL($0))])
-                              HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
-                          HiveProject(d_date_sk=[$0])
-                            HiveFilter(condition=[AND(=($6, 2000), =($8, 11), IS NOT NULL($0))])
-                              HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-                      HiveProject(i_item_sk=[$0])
-                        HiveJoin(condition=[AND(=($1, $4), =($2, $5), =($3, $6))], joinType=[inner], algorithm=[none], cost=[not available])
+                HiveFilter(condition=[IS NOT NULL($3)])
+                  HiveAggregate(group=[{0, 1, 2}], agg#0=[sum($3)], agg#1=[count()])
+                    HiveProject(i_brand_id=[$1], i_class_id=[$2], i_category_id=[$3], $f3=[*(CAST($6):DECIMAL(10, 0), $7)])
+                      HiveSemiJoin(condition=[=($5, $9)], joinType=[inner])
+                        HiveJoin(condition=[=($5, $0)], joinType=[inner], algorithm=[none], cost=[not available])
                           HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
-                            HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11), IS NOT NULL($0))])
+                            HiveFilter(condition=[IS NOT NULL($0)])
                               HiveTableScan(table=[[default, item]], table:alias=[item])
-                          HiveProject($f0=[$0], $f1=[$1], $f2=[$2])
-                            HiveFilter(condition=[=($3, 3)])
-                              HiveAggregate(group=[{0, 1, 2}], agg#0=[count($3)])
-                                HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3])
-                                  HiveUnion(all=[true])
-                                    HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3])
-                                      HiveAggregate(group=[{4, 5, 6}], agg#0=[count()])
-                                        HiveJoin(condition=[=($1, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                                          HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-                                            HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2])
-                                              HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($0))])
-                                                HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                                            HiveProject(d_date_sk=[$0])
-                                              HiveFilter(condition=[AND(BETWEEN(false, $6, 1999, 2001), IS NOT NULL($0))])
-                                                HiveTableScan(table=[[default, date_dim]], table:alias=[d1])
-                                          HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
-                                            HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11))])
-                                              HiveTableScan(table=[[default, item]], table:alias=[iss])
-                                    HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3])
-                                      HiveAggregate(group=[{4, 5, 6}], agg#0=[count()])
-                                        HiveJoin(condition=[=($1, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                                          HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-                                            HiveProject(cs_sold_date_sk=[$0], cs_item_sk=[$15])
-                                              HiveFilter(condition=[AND(IS NOT NULL($15), IS NOT NULL($0))])
-                                                HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
-                                            HiveProject(d_date_sk=[$0])
-                                              HiveFilter(condition=[AND(BETWEEN(false, $6, 1999, 2001), IS NOT NULL($0))])
-                                                HiveTableScan(table=[[default, date_dim]], table:alias=[d2])
-                                          HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
-                                            HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11))])
-                                              HiveTableScan(table=[[default, item]], table:alias=[ics])
-                                    HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3])
-                                      HiveAggregate(group=[{4, 5, 6}], agg#0=[count()])
-                                        HiveJoin(condition=[=($1, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                                          HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-                                            HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$3])
-                                              HiveFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($0))])
-                                                HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
-                                            HiveProject(d_date_sk=[$0])
-                                              HiveFilter(condition=[AND(BETWEEN(false, $6, 1999, 2001), IS NOT NULL($0))])
-                                                HiveTableScan(table=[[default, date_dim]], table:alias=[d3])
-                                          HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
-                                            HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11))])
-                                              HiveTableScan(table=[[default, item]], table:alias=[iws])
+                          HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available])
+                            HiveProject(cs_sold_date_sk=[$0], cs_item_sk=[$15], cs_quantity=[$18], cs_list_price=[$20])
+                              HiveFilter(condition=[AND(IS NOT NULL($15), IS NOT NULL($0))])
+                                HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
+                            HiveProject(d_date_sk=[$0])
+                              HiveFilter(condition=[AND(=($6, 2000), =($8, 11), IS NOT NULL($0))])
+                                HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+                        HiveProject(i_item_sk=[$0])
+                          HiveJoin(condition=[AND(=($1, $4), =($2, $5), =($3, $6))], joinType=[inner], algorithm=[none], cost=[not available])
+                            HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
+                              HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11), IS NOT NULL($0))])
+                                HiveTableScan(table=[[default, item]], table:alias=[item])
+                            HiveProject($f0=[$0], $f1=[$1], $f2=[$2])
+                              HiveFilter(condition=[=($3, 3)])
+                                HiveAggregate(group=[{0, 1, 2}], agg#0=[count($3)])
+                                  HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3])
+                                    HiveUnion(all=[true])
+                                      HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3])
+                                        HiveAggregate(group=[{4, 5, 6}], agg#0=[count()])
+                                          HiveJoin(condition=[=($1, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                                            HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+                                              HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2])
+                                                HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($0))])
+                                                  HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                                              HiveProject(d_date_sk=[$0])
+                                                HiveFilter(condition=[AND(BETWEEN(false, $6, 1999, 2001), IS NOT NULL($0))])
+                                                  HiveTableScan(table=[[default, date_dim]], table:alias=[d1])
+                                            HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
+                                              HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11))])
+                                                HiveTableScan(table=[[default, item]], table:alias=[iss])
+                                      HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3])
+                                        HiveAggregate(group=[{4, 5, 6}], agg#0=[count()])
+                                          HiveJoin(condition=[=($1, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                                            HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+                                              HiveProject(cs_sold_date_sk=[$0], cs_item_sk=[$15])
+                                                HiveFilter(condition=[AND(IS NOT NULL($15), IS NOT NULL($0))])
+                                                  HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
+                                              HiveProject(d_date_sk=[$0])
+                                                HiveFilter(condition=[AND(BETWEEN(false, $6, 1999, 2001), IS NOT NULL($0))])
+                                                  HiveTableScan(table=[[default, date_dim]], table:alias=[d2])
+                                            HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
+                                              HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11))])
+                                                HiveTableScan(table=[[default, item]], table:alias=[ics])
+                                      HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3])
+                                        HiveAggregate(group=[{4, 5, 6}], agg#0=[count()])
+                                          HiveJoin(condition=[=($1, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                                            HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+                                              HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$3])
+                                                HiveFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($0))])
+                                                  HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
+                                              HiveProject(d_date_sk=[$0])
+                                                HiveFilter(condition=[AND(BETWEEN(false, $6, 1999, 2001), IS NOT NULL($0))])
+                                                  HiveTableScan(table=[[default, date_dim]], table:alias=[d3])
+                                            HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
+                                              HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11))])
+                                                HiveTableScan(table=[[default, item]], table:alias=[iws])
               HiveProject($f0=[/($0, $1)])
-                HiveAggregate(group=[{}], agg#0=[sum($0)], agg#1=[count($0)])
-                  HiveProject($f0=[*(CAST($0):DECIMAL(10, 0), $1)])
-                    HiveUnion(all=[true])
-                      HiveProject(quantity=[$1], list_price=[$2])
-                        HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                          HiveProject(ss_sold_date_sk=[$0], ss_quantity=[$10], ss_list_price=[$12])
-                            HiveFilter(condition=[IS NOT NULL($0)])
-                              HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                          HiveProject(d_date_sk=[$0])
-                            HiveFilter(condition=[AND(BETWEEN(false, $6, 1999, 2001), IS NOT NULL($0))])
-                              HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-                      HiveProject(quantity=[$1], list_price=[$2])
-                        HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                          HiveProject(cs_sold_date_sk=[$0], cs_quantity=[$18], cs_list_price=[$20])
-                            HiveFilter(condition=[IS NOT NULL($0)])
-                              HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
-                          HiveProject(d_date_sk=[$0])
-                            HiveFilter(condition=[AND(BETWEEN(false, $6, 1998, 2000), IS NOT NULL($0))])
-                              HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-                      HiveProject(quantity=[$1], list_price=[$2])
-                        HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                          HiveProject(ws_sold_date_sk=[$0], ws_quantity=[$18], ws_list_price=[$20])
-                            HiveFilter(condition=[IS NOT NULL($0)])
-                              HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
-                          HiveProject(d_date_sk=[$0])
-                            HiveFilter(condition=[AND(BETWEEN(false, $6, 1998, 2000), IS NOT NULL($0))])
-                              HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+                HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($1))])
+                  HiveProject($f0=[$0], $f1=[$1])
+                    HiveAggregate(group=[{}], agg#0=[sum($0)], agg#1=[count($0)])
+                      HiveProject($f0=[*(CAST($0):DECIMAL(10, 0), $1)])
+                        HiveUnion(all=[true])
+                          HiveProject(quantity=[$1], list_price=[$2])
+                            HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                              HiveProject(ss_sold_date_sk=[$0], ss_quantity=[$10], ss_list_price=[$12])
+                                HiveFilter(condition=[IS NOT NULL($0)])
+                                  HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                              HiveProject(d_date_sk=[$0])
+                                HiveFilter(condition=[AND(BETWEEN(false, $6, 1999, 2001), IS NOT NULL($0))])
+                                  HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+                          HiveProject(quantity=[$1], list_price=[$2])
+                            HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                              HiveProject(cs_sold_date_sk=[$0], cs_quantity=[$18], cs_list_price=[$20])
+                                HiveFilter(condition=[IS NOT NULL($0)])
+                                  HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
+                              HiveProject(d_date_sk=[$0])
+                                HiveFilter(condition=[AND(BETWEEN(false, $6, 1998, 2000), IS NOT NULL($0))])
+                                  HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+                          HiveProject(quantity=[$1], list_price=[$2])
+                            HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                              HiveProject(ws_sold_date_sk=[$0], ws_quantity=[$18], ws_list_price=[$20])
+                                HiveFilter(condition=[IS NOT NULL($0)])
+                                  HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
+                              HiveProject(d_date_sk=[$0])
+                                HiveFilter(condition=[AND(BETWEEN(false, $6, 1998, 2000), IS NOT NULL($0))])
+                                  HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
           HiveProject(channel=[_UTF-16LE'web':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"], i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], sales=[$3], number_sales=[$4])
             HiveJoin(condition=[>($3, $5)], joinType=[inner], algorithm=[none], cost=[not available])
               HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3], $f4=[$4])
-                HiveAggregate(group=[{0, 1, 2}], agg#0=[sum($3)], agg#1=[count()])
-                  HiveProject(i_brand_id=[$1], i_class_id=[$2], i_category_id=[$3], $f3=[*(CAST($6):DECIMAL(10, 0), $7)])
-                    HiveSemiJoin(condition=[=($5, $9)], joinType=[inner])
-                      HiveJoin(condition=[=($5, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                        HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
-                          HiveFilter(condition=[IS NOT NULL($0)])
-                            HiveTableScan(table=[[default, item]], table:alias=[item])
-                        HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available])
-                          HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$3], ws_quantity=[$18], ws_list_price=[$20])
-                            HiveFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($0))])
-                              HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
-                          HiveProject(d_date_sk=[$0])
-                            HiveFilter(condition=[AND(=($6, 2000), =($8, 11), IS NOT NULL($0))])
-                              HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-                      HiveProject(i_item_sk=[$0])
-                        HiveJoin(condition=[AND(=($1, $4), =($2, $5), =($3, $6))], joinType=[inner], algorithm=[none], cost=[not available])
+                HiveFilter(condition=[IS NOT NULL($3)])
+                  HiveAggregate(group=[{0, 1, 2}], agg#0=[sum($3)], agg#1=[count()])
+                    HiveProject(i_brand_id=[$1], i_class_id=[$2], i_category_id=[$3], $f3=[*(CAST($6):DECIMAL(10, 0), $7)])
+                      HiveSemiJoin(condition=[=($5, $9)], joinType=[inner])
+                        HiveJoin(condition=[=($5, $0)], joinType=[inner], algorithm=[none], cost=[not available])
                           HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
-                            HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11), IS NOT NULL($0))])
+                            HiveFilter(condition=[IS NOT NULL($0)])
                               HiveTableScan(table=[[default, item]], table:alias=[item])
-                          HiveProject($f0=[$0], $f1=[$1], $f2=[$2])
-                            HiveFilter(condition=[=($3, 3)])
-                              HiveAggregate(group=[{0, 1, 2}], agg#0=[count($3)])
-                                HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3])
-                                  HiveUnion(all=[true])
-                                    HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3])
-                                      HiveAggregate(group=[{4, 5, 6}], agg#0=[count()])
-                                        HiveJoin(condition=[=($1, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                                          HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-                                            HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2])
-                                              HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($0))])
-                                                HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                                            HiveProject(d_date_sk=[$0])
-                                              HiveFilter(condition=[AND(BETWEEN(false, $6, 1999, 2001), IS NOT NULL($0))])
-                                                HiveTableScan(table=[[default, date_dim]], table:alias=[d1])
-                                          HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
-                                            HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11))])
-                                              HiveTableScan(table=[[default, item]], table:alias=[iss])
-                                    HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3])
-                                      HiveAggregate(group=[{4, 5, 6}], agg#0=[count()])
-                                        HiveJoin(condition=[=($1, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                                          HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-                                            HiveProject(cs_sold_date_sk=[$0], cs_item_sk=[$15])
-                                              HiveFilter(condition=[AND(IS NOT NULL($15), IS NOT NULL($0))])
-                                                HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
-                                            HiveProject(d_date_sk=[$0])
-                                              HiveFilter(condition=[AND(BETWEEN(false, $6, 1999, 2001), IS NOT NULL($0))])
-                                                HiveTableScan(table=[[default, date_dim]], table:alias=[d2])
-                                          HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
-                                            HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11))])
-                                              HiveTableScan(table=[[default, item]], table:alias=[ics])
-                                    HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3])
-                                      HiveAggregate(group=[{4, 5, 6}], agg#0=[count()])
-                                        HiveJoin(condition=[=($1, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                                          HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-                                            HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$3])
-                                              HiveFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($0))])
-                                                HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
-                                            HiveProject(d_date_sk=[$0])
-                                              HiveFilter(condition=[AND(BETWEEN(false, $6, 1999, 2001), IS NOT NULL($0))])
-                                                HiveTableScan(table=[[default, date_dim]], table:alias=[d3])
-                                          HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
-                                            HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11))])
-                                              HiveTableScan(table=[[default, item]], table:alias=[iws])
+                          HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available])
+                            HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$3], ws_quantity=[$18], ws_list_price=[$20])
+                              HiveFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($0))])
+                                HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
+                            HiveProject(d_date_sk=[$0])
+                              HiveFilter(condition=[AND(=($6, 2000), =($8, 11), IS NOT NULL($0))])
+                                HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+                        HiveProject(i_item_sk=[$0])
+                          HiveJoin(condition=[AND(=($1, $4), =($2, $5), =($3, $6))], joinType=[inner], algorithm=[none], cost=[not available])
+                            HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
+                              HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11), IS NOT NULL($0))])
+                                HiveTableScan(table=[[default, item]], table:alias=[item])
+                            HiveProject($f0=[$0], $f1=[$1], $f2=[$2])
+                              HiveFilter(condition=[=($3, 3)])
+                                HiveAggregate(group=[{0, 1, 2}], agg#0=[count($3)])
+                                  HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3])
+                                    HiveUnion(all=[true])
+                                      HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3])
+                                        HiveAggregate(group=[{4, 5, 6}], agg#0=[count()])
+                                          HiveJoin(condition=[=($1, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                                            HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+                                              HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2])
+                                                HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($0))])
+                                                  HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                                              HiveProject(d_date_sk=[$0])
+                                                HiveFilter(condition=[AND(BETWEEN(false, $6, 1999, 2001), IS NOT NULL($0))])
+                                                  HiveTableScan(table=[[default, date_dim]], table:alias=[d1])
+                                            HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
+                                              HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11))])
+                                                HiveTableScan(table=[[default, item]], table:alias=[iss])
+                                      HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3])
+                                        HiveAggregate(group=[{4, 5, 6}], agg#0=[count()])
+                                          HiveJoin(condition=[=($1, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                                            HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+                                              HiveProject(cs_sold_date_sk=[$0], cs_item_sk=[$15])
+                                                HiveFilter(condition=[AND(IS NOT NULL($15), IS NOT NULL($0))])
+                                                  HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
+                                              HiveProject(d_date_sk=[$0])
+                                                HiveFilter(condition=[AND(BETWEEN(false, $6, 1999, 2001), IS NOT NULL($0))])
+                                                  HiveTableScan(table=[[default, date_dim]], table:alias=[d2])
+                                            HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
+                                              HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11))])
+                                                HiveTableScan(table=[[default, item]], table:alias=[ics])
+                                      HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3])
+                                        HiveAggregate(group=[{4, 5, 6}], agg#0=[count()])
+                                          HiveJoin(condition=[=($1, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                                            HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+                                              HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$3])
+                                                HiveFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($0))])
+                                                  HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
+                                              HiveProject(d_date_sk=[$0])
+                                                HiveFilter(condition=[AND(BETWEEN(false, $6, 1999, 2001), IS NOT NULL($0))])
+                                                  HiveTableScan(table=[[default, date_dim]], table:alias=[d3])
+                                            HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
+                                              HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11))])
+                                                HiveTableScan(table=[[default, item]], table:alias=[iws])
               HiveProject($f0=[/($0, $1)])
-                HiveAggregate(group=[{}], agg#0=[sum($0)], agg#1=[count($0)])
-                  HiveProject($f0=[*(CAST($0):DECIMAL(10, 0), $1)])
-                    HiveUnion(all=[true])
-                      HiveProject(quantity=[$1], list_price=[$2])
-                        HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                          HiveProject(ss_sold_date_sk=[$0], ss_quantity=[$10], ss_list_price=[$12])
-                            HiveFilter(condition=[IS NOT NULL($0)])
-                              HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                          HiveProject(d_date_sk=[$0])
-                            HiveFilter(condition=[AND(BETWEEN(false, $6, 1999, 2001), IS NOT NULL($0))])
-                              HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-                      HiveProject(quantity=[$1], list_price=[$2])
-                        HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                          HiveProject(cs_sold_date_sk=[$0], cs_quantity=[$18], cs_list_price=[$20])
-                            HiveFilter(condition=[IS NOT NULL($0)])
-                              HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
-                          HiveProject(d_date_sk=[$0])
-                            HiveFilter(condition=[AND(BETWEEN(false, $6, 1998, 2000), IS NOT NULL($0))])
-                              HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-                      HiveProject(quantity=[$1], list_price=[$2])
-                        HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                          HiveProject(ws_sold_date_sk=[$0], ws_quantity=[$18], ws_list_price=[$20])
-                            HiveFilter(condition=[IS NOT NULL($0)])
-                              HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
-                          HiveProject(d_date_sk=[$0])
-                            HiveFilter(condition=[AND(BETWEEN(false, $6, 1998, 2000), IS NOT NULL($0))])
-                              HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+                HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($1))])
+                  HiveProject($f0=[$0], $f1=[$1])
+                    HiveAggregate(group=[{}], agg#0=[sum($0)], agg#1=[count($0)])
+                      HiveProject($f0=[*(CAST($0):DECIMAL(10, 0), $1)])
+                        HiveUnion(all=[true])
+                          HiveProject(quantity=[$1], list_price=[$2])
+                            HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                              HiveProject(ss_sold_date_sk=[$0], ss_quantity=[$10], ss_list_price=[$12])
+                                HiveFilter(condition=[IS NOT NULL($0)])
+                                  HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                              HiveProject(d_date_sk=[$0])
+                                HiveFilter(condition=[AND(BETWEEN(false, $6, 1999, 2001), IS NOT NULL($0))])
+                                  HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+                          HiveProject(quantity=[$1], list_price=[$2])
+                            HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                              HiveProject(cs_sold_date_sk=[$0], cs_quantity=[$18], cs_list_price=[$20])
+                                HiveFilter(condition=[IS NOT NULL($0)])
+                                  HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
+                              HiveProject(d_date_sk=[$0])
+                                HiveFilter(condition=[AND(BETWEEN(false, $6, 1998, 2000), IS NOT NULL($0))])
+                                  HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+                          HiveProject(quantity=[$1], list_price=[$2])
+                            HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                              HiveProject(ws_sold_date_sk=[$0], ws_quantity=[$18], ws_list_price=[$20])
+                                HiveFilter(condition=[IS NOT NULL($0)])
+                                  HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
+                              HiveProject(d_date_sk=[$0])
+                                HiveFilter(condition=[AND(BETWEEN(false, $6, 1998, 2000), IS NOT NULL($0))])
+                                  HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/cbo_query23.q.out b/ql/src/test/results/clientpositive/perf/tez/cbo_query23.q.out
index d41b197..f004144 100644
--- a/ql/src/test/results/clientpositive/perf/tez/cbo_query23.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/cbo_query23.q.out
@@ -1,5 +1,5 @@
-Warning: Shuffle Join MERGEJOIN[448][tables = [$hdt$_3, $hdt$_4]] in Stage 'Reducer 25' is a cross product
-Warning: Shuffle Join MERGEJOIN[450][tables = [$hdt$_3, $hdt$_4]] in Stage 'Reducer 30' is a cross product
+Warning: Shuffle Join MERGEJOIN[454][tables = [$hdt$_3, $hdt$_4]] in Stage 'Reducer 25' is a cross product
+Warning: Shuffle Join MERGEJOIN[456][tables = [$hdt$_3, $hdt$_4]] in Stage 'Reducer 30' is a cross product
 PREHOOK: query: explain cbo
 with frequent_ss_items as 
  (select substr(i_item_desc,1,30) itemdesc,i_item_sk item_sk,d_date solddate,count(*) cnt
@@ -118,9 +118,9 @@ POSTHOOK: Input: default@web_sales
 POSTHOOK: Output: hdfs://### HDFS PATH ###
 CBO PLAN:
 HiveAggregate(group=[{}], agg#0=[sum($0)])
-  HiveProject($f0=[$0])
+  HiveProject(sales=[$0])
     HiveUnion(all=[true])
-      HiveProject($f0=[*(CAST($4):DECIMAL(10, 0), $5)])
+      HiveProject(sales=[*(CAST($4):DECIMAL(10, 0), $5)])
         HiveSemiJoin(condition=[=($2, $7)], joinType=[inner])
           HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
             HiveProject(i_item_sk=[$0])
@@ -149,30 +149,33 @@ HiveAggregate(group=[{}], agg#0=[sum($0)])
           HiveProject(c_customer_sk=[$0])
             HiveJoin(condition=[>($1, $2)], joinType=[inner], algorithm=[none], cost=[not available])
               HiveProject(c_customer_sk=[$0], $f1=[$1])
-                HiveAggregate(group=[{2}], agg#0=[sum($1)])
-                  HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-                    HiveProject(ss_customer_sk=[$3], *=[*(CAST($10):DECIMAL(10, 0), $13)])
-                      HiveFilter(condition=[IS NOT NULL($3)])
-                        HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                    HiveProject(c_customer_sk=[$0])
-                      HiveFilter(condition=[IS NOT NULL($0)])
-                        HiveTableScan(table=[[default, customer]], table:alias=[customer])
+                HiveFilter(condition=[IS NOT NULL($1)])
+                  HiveAggregate(group=[{2}], agg#0=[sum($1)])
+                    HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+                      HiveProject(ss_customer_sk=[$3], *=[*(CAST($10):DECIMAL(10, 0), $13)])
+                        HiveFilter(condition=[IS NOT NULL($3)])
+                          HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                      HiveProject(c_customer_sk=[$0])
+                        HiveFilter(condition=[IS NOT NULL($0)])
+                          HiveTableScan(table=[[default, customer]], table:alias=[customer])
               HiveProject(*=[*(0.95:DECIMAL(3, 2), $0)])
-                HiveAggregate(group=[{}], agg#0=[max($1)])
-                  HiveProject(c_customer_sk=[$0], $f1=[$1])
-                    HiveAggregate(group=[{0}], agg#0=[sum($3)])
-                      HiveJoin(condition=[=($2, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                        HiveProject(c_customer_sk=[$0])
-                          HiveFilter(condition=[IS NOT NULL($0)])
-                            HiveTableScan(table=[[default, customer]], table:alias=[customer])
-                        HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                          HiveProject(ss_sold_date_sk=[$0], ss_customer_sk=[$3], *=[*(CAST($10):DECIMAL(10, 0), $13)])
-                            HiveFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($0))])
-                              HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                          HiveProject(d_date_sk=[$0])
-                            HiveFilter(condition=[AND(IN($6, 1999, 2000, 2001, 2002), IS NOT NULL($0))])
-                              HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-      HiveProject($f0=[*(CAST($4):DECIMAL(10, 0), $5)])
+                HiveFilter(condition=[IS NOT NULL($0)])
+                  HiveProject($f0=[$0])
+                    HiveAggregate(group=[{}], agg#0=[max($1)])
+                      HiveProject(c_customer_sk=[$0], $f1=[$1])
+                        HiveAggregate(group=[{0}], agg#0=[sum($3)])
+                          HiveJoin(condition=[=($2, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+                            HiveProject(c_customer_sk=[$0])
+                              HiveFilter(condition=[IS NOT NULL($0)])
+                                HiveTableScan(table=[[default, customer]], table:alias=[customer])
+                            HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                              HiveProject(ss_sold_date_sk=[$0], ss_customer_sk=[$3], *=[*(CAST($10):DECIMAL(10, 0), $13)])
+                                HiveFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($0))])
+                                  HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                              HiveProject(d_date_sk=[$0])
+                                HiveFilter(condition=[AND(IN($6, 1999, 2000, 2001, 2002), IS NOT NULL($0))])
+                                  HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+      HiveProject(sales=[*(CAST($4):DECIMAL(10, 0), $5)])
         HiveSemiJoin(condition=[=($3, $7)], joinType=[inner])
           HiveJoin(condition=[=($2, $0)], joinType=[inner], algorithm=[none], cost=[not available])
             HiveProject(i_item_sk=[$0])
@@ -201,27 +204,30 @@ HiveAggregate(group=[{}], agg#0=[sum($0)])
           HiveProject(c_customer_sk=[$0])
             HiveJoin(condition=[>($1, $2)], joinType=[inner], algorithm=[none], cost=[not available])
               HiveProject(c_customer_sk=[$0], $f1=[$1])
-                HiveAggregate(group=[{2}], agg#0=[sum($1)])
-                  HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-                    HiveProject(ss_customer_sk=[$3], *=[*(CAST($10):DECIMAL(10, 0), $13)])
-                      HiveFilter(condition=[IS NOT NULL($3)])
-                        HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                    HiveProject(c_customer_sk=[$0])
-                      HiveFilter(condition=[IS NOT NULL($0)])
-                        HiveTableScan(table=[[default, customer]], table:alias=[customer])
+                HiveFilter(condition=[IS NOT NULL($1)])
+                  HiveAggregate(group=[{2}], agg#0=[sum($1)])
+                    HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+                      HiveProject(ss_customer_sk=[$3], *=[*(CAST($10):DECIMAL(10, 0), $13)])
+                        HiveFilter(condition=[IS NOT NULL($3)])
+                          HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                      HiveProject(c_customer_sk=[$0])
+                        HiveFilter(condition=[IS NOT NULL($0)])
+                          HiveTableScan(table=[[default, customer]], table:alias=[customer])
               HiveProject(*=[*(0.95:DECIMAL(3, 2), $0)])
-                HiveAggregate(group=[{}], agg#0=[max($1)])
-                  HiveProject(c_customer_sk=[$0], $f1=[$1])
-                    HiveAggregate(group=[{0}], agg#0=[sum($3)])
-                      HiveJoin(condition=[=($2, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                        HiveProject(c_customer_sk=[$0])
-                          HiveFilter(condition=[IS NOT NULL($0)])
-                            HiveTableScan(table=[[default, customer]], table:alias=[customer])
-                        HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                          HiveProject(ss_sold_date_sk=[$0], ss_customer_sk=[$3], *=[*(CAST($10):DECIMAL(10, 0), $13)])
-                            HiveFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($0))])
-                              HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                          HiveProject(d_date_sk=[$0])
-                            HiveFilter(condition=[AND(IN($6, 1999, 2000, 2001, 2002), IS NOT NULL($0))])
-                              HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+                HiveFilter(condition=[IS NOT NULL($0)])
+                  HiveProject($f0=[$0])
+                    HiveAggregate(group=[{}], agg#0=[max($1)])
+                      HiveProject(c_customer_sk=[$0], $f1=[$1])
+                        HiveAggregate(group=[{0}], agg#0=[sum($3)])
+                          HiveJoin(condition=[=($2, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+                            HiveProject(c_customer_sk=[$0])
+                              HiveFilter(condition=[IS NOT NULL($0)])
+                                HiveTableScan(table=[[default, customer]], table:alias=[customer])
+                            HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                              HiveProject(ss_sold_date_sk=[$0], ss_customer_sk=[$3], *=[*(CAST($10):DECIMAL(10, 0), $13)])
+                                HiveFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($0))])
+                                  HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                              HiveProject(d_date_sk=[$0])
+                                HiveFilter(condition=[AND(IN($6, 1999, 2000, 2001, 2002), IS NOT NULL($0))])
+                                  HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/cbo_query24.q.out b/ql/src/test/results/clientpositive/perf/tez/cbo_query24.q.out
index 511f8fb..6c4426c 100644
--- a/ql/src/test/results/clientpositive/perf/tez/cbo_query24.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/cbo_query24.q.out
@@ -1,4 +1,4 @@
-Warning: Shuffle Join MERGEJOIN[301][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 6' is a cross product
+Warning: Shuffle Join MERGEJOIN[305][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 6' is a cross product
 PREHOOK: query: explain cbo
 with ssales as
 (select c_last_name
@@ -116,59 +116,63 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 CBO PLAN:
 HiveProject($f0=[$0], $f1=[$1], $f2=[$2], $f3=[$3])
   HiveJoin(condition=[>($3, $4)], joinType=[inner], algorithm=[none], cost=[not available])
-    HiveProject(c_last_name=[$1], c_first_name=[$0], s_store_name=[$2], $f3=[$3])
-      HiveAggregate(group=[{4, 5, 7}], agg#0=[sum($9)])
-        HiveProject(i_current_price=[$0], i_size=[$1], i_units=[$2], i_manager_id=[$3], c_first_name=[$4], c_last_name=[$5], ca_state=[$6], s_store_name=[$7], s_state=[$8], $f9=[$9])
-          HiveAggregate(group=[{8, 9, 10, 11, 14, 15, 18, 22, 23}], agg#0=[sum($6)])
-            HiveJoin(condition=[AND(=($5, $1), =($2, $0))], joinType=[inner], algorithm=[none], cost=[not available])
-              HiveProject(sr_item_sk=[$2], sr_ticket_number=[$9])
-                HiveFilter(condition=[AND(IS NOT NULL($9), IS NOT NULL($2))])
-                  HiveTableScan(table=[[default, store_returns]], table:alias=[store_returns])
-              HiveJoin(condition=[AND(=($1, $10), =($2, $19))], joinType=[inner], algorithm=[none], cost=[not available])
-                HiveJoin(condition=[=($0, $5)], joinType=[inner], algorithm=[none], cost=[not available])
-                  HiveProject(ss_item_sk=[$2], ss_customer_sk=[$3], ss_store_sk=[$7], ss_ticket_number=[$9], ss_sales_price=[$13])
-                    HiveFilter(condition=[AND(IS NOT NULL($9), IS NOT NULL($2), IS NOT NULL($7), IS NOT NULL($3))])
-                      HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                  HiveProject(i_item_sk=[$0], i_current_price=[$5], i_size=[$15], i_units=[$18], i_manager_id=[$20])
-                    HiveFilter(condition=[AND(=($17, _UTF-16LE'orchid'), IS NOT NULL($0))])
-                      HiveTableScan(table=[[default, item]], table:alias=[item])
-                HiveProject(c_customer_sk=[$0], c_current_addr_sk=[$1], c_first_name=[$2], c_last_name=[$3], c_birth_country=[$4], ca_address_sk=[$5], ca_state=[$6], ca_zip=[$7], UPPER=[$8], s_store_sk=[$9], s_store_name=[$10], s_state=[$11], s_zip=[$12])
-                  HiveJoin(condition=[AND(=($1, $5), <>($4, $8))], joinType=[inner], algorithm=[none], cost=[not available])
-                    HiveProject(c_customer_sk=[$0], c_current_addr_sk=[$4], c_first_name=[$8], c_last_name=[$9], c_birth_country=[$14])
-                      HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($4))])
-                        HiveTableScan(table=[[default, customer]], table:alias=[customer])
-                    HiveJoin(condition=[=($7, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-                      HiveProject(ca_address_sk=[$0], ca_state=[$8], ca_zip=[$9], UPPER=[UPPER($10)])
-                        HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($9))])
-                          HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
-                      HiveProject(s_store_sk=[$0], s_store_name=[$5], s_state=[$24], s_zip=[$25])
-                        HiveFilter(condition=[AND(=($10, 7), IS NOT NULL($0), IS NOT NULL($25))])
-                          HiveTableScan(table=[[default, store]], table:alias=[store])
+    HiveProject(c_last_name=[$0], c_first_name=[$1], s_store_name=[$2], $f3=[$3])
+      HiveFilter(condition=[IS NOT NULL($3)])
+        HiveProject(c_last_name=[$1], c_first_name=[$0], s_store_name=[$2], $f3=[$3])
+          HiveAggregate(group=[{4, 5, 7}], agg#0=[sum($9)])
+            HiveProject(i_current_price=[$0], i_size=[$1], i_units=[$2], i_manager_id=[$3], c_first_name=[$4], c_last_name=[$5], ca_state=[$6], s_store_name=[$7], s_state=[$8], $f9=[$9])
+              HiveAggregate(group=[{8, 9, 10, 11, 14, 15, 18, 22, 23}], agg#0=[sum($6)])
+                HiveJoin(condition=[AND(=($5, $1), =($2, $0))], joinType=[inner], algorithm=[none], cost=[not available])
+                  HiveProject(sr_item_sk=[$2], sr_ticket_number=[$9])
+                    HiveFilter(condition=[AND(IS NOT NULL($9), IS NOT NULL($2))])
+                      HiveTableScan(table=[[default, store_returns]], table:alias=[store_returns])
+                  HiveJoin(condition=[AND(=($1, $10), =($2, $19))], joinType=[inner], algorithm=[none], cost=[not available])
+                    HiveJoin(condition=[=($0, $5)], joinType=[inner], algorithm=[none], cost=[not available])
+                      HiveProject(ss_item_sk=[$2], ss_customer_sk=[$3], ss_store_sk=[$7], ss_ticket_number=[$9], ss_sales_price=[$13])
+                        HiveFilter(condition=[AND(IS NOT NULL($9), IS NOT NULL($2), IS NOT NULL($7), IS NOT NULL($3))])
+                          HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                      HiveProject(i_item_sk=[$0], i_current_price=[$5], i_size=[$15], i_units=[$18], i_manager_id=[$20])
+                        HiveFilter(condition=[AND(=($17, _UTF-16LE'orchid'), IS NOT NULL($0))])
+                          HiveTableScan(table=[[default, item]], table:alias=[item])
+                    HiveProject(c_customer_sk=[$0], c_current_addr_sk=[$1], c_first_name=[$2], c_last_name=[$3], c_birth_country=[$4], ca_address_sk=[$5], ca_state=[$6], ca_zip=[$7], UPPER=[$8], s_store_sk=[$9], s_store_name=[$10], s_state=[$11], s_zip=[$12])
+                      HiveJoin(condition=[AND(=($1, $5), <>($4, $8))], joinType=[inner], algorithm=[none], cost=[not available])
+                        HiveProject(c_customer_sk=[$0], c_current_addr_sk=[$4], c_first_name=[$8], c_last_name=[$9], c_birth_country=[$14])
+                          HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($4))])
+                            HiveTableScan(table=[[default, customer]], table:alias=[customer])
+                        HiveJoin(condition=[=($7, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+                          HiveProject(ca_address_sk=[$0], ca_state=[$8], ca_zip=[$9], UPPER=[UPPER($10)])
+                            HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($9))])
+                              HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
+                          HiveProject(s_store_sk=[$0], s_store_name=[$5], s_state=[$24], s_zip=[$25])
+                            HiveFilter(condition=[AND(=($10, 7), IS NOT NULL($0), IS NOT NULL($25))])
+                              HiveTableScan(table=[[default, store]], table:alias=[store])
     HiveProject(_o__c0=[*(0.05:DECIMAL(3, 2), /($0, $1))])
-      HiveAggregate(group=[{}], agg#0=[sum($10)], agg#1=[count($10)])
-        HiveProject(c_first_name=[$0], c_last_name=[$1], ca_state=[$2], s_store_name=[$3], s_state=[$4], i_current_price=[$5], i_size=[$6], i_color=[$7], i_units=[$8], i_manager_id=[$9], $f10=[$10])
-          HiveAggregate(group=[{7, 8, 11, 15, 16, 19, 20, 21, 22, 23}], agg#0=[sum($4)])
-            HiveJoin(condition=[AND(=($3, $25), =($0, $24))], joinType=[inner], algorithm=[none], cost=[not available])
-              HiveJoin(condition=[=($0, $18)], joinType=[inner], algorithm=[none], cost=[not available])
-                HiveJoin(condition=[AND(=($1, $5), =($2, $14))], joinType=[inner], algorithm=[none], cost=[not available])
-                  HiveProject(ss_item_sk=[$2], ss_customer_sk=[$3], ss_store_sk=[$7], ss_ticket_number=[$9], ss_sales_price=[$13])
-                    HiveFilter(condition=[AND(IS NOT NULL($9), IS NOT NULL($2), IS NOT NULL($7), IS NOT NULL($3))])
-                      HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                  HiveJoin(condition=[AND(=($1, $5), <>($4, $8))], joinType=[inner], algorithm=[none], cost=[not available])
-                    HiveProject(c_customer_sk=[$0], c_current_addr_sk=[$4], c_first_name=[$8], c_last_name=[$9], c_birth_country=[$14])
-                      HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($4))])
-                        HiveTableScan(table=[[default, customer]], table:alias=[customer])
-                    HiveJoin(condition=[=($7, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-                      HiveProject(ca_address_sk=[$0], ca_state=[$8], ca_zip=[$9], UPPER=[UPPER($10)])
-                        HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($9))])
-                          HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
-                      HiveProject(s_store_sk=[$0], s_store_name=[$5], s_state=[$24], s_zip=[$25])
-                        HiveFilter(condition=[AND(=($10, 7), IS NOT NULL($0), IS NOT NULL($25))])
-                          HiveTableScan(table=[[default, store]], table:alias=[store])
-                HiveProject(i_item_sk=[$0], i_current_price=[$5], i_size=[$15], i_color=[$17], i_units=[$18], i_manager_id=[$20])
-                  HiveFilter(condition=[IS NOT NULL($0)])
-                    HiveTableScan(table=[[default, item]], table:alias=[item])
-              HiveProject(sr_item_sk=[$2], sr_ticket_number=[$9])
-                HiveFilter(condition=[AND(IS NOT NULL($9), IS NOT NULL($2))])
-                  HiveTableScan(table=[[default, store_returns]], table:alias=[store_returns])
+      HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($1))])
+        HiveProject($f0=[$0], $f1=[$1])
+          HiveAggregate(group=[{}], agg#0=[sum($10)], agg#1=[count($10)])
+            HiveProject(c_first_name=[$0], c_last_name=[$1], ca_state=[$2], s_store_name=[$3], s_state=[$4], i_current_price=[$5], i_size=[$6], i_color=[$7], i_units=[$8], i_manager_id=[$9], $f10=[$10])
+              HiveAggregate(group=[{7, 8, 11, 15, 16, 19, 20, 21, 22, 23}], agg#0=[sum($4)])
+                HiveJoin(condition=[AND(=($3, $25), =($0, $24))], joinType=[inner], algorithm=[none], cost=[not available])
+                  HiveJoin(condition=[=($0, $18)], joinType=[inner], algorithm=[none], cost=[not available])
+                    HiveJoin(condition=[AND(=($1, $5), =($2, $14))], joinType=[inner], algorithm=[none], cost=[not available])
+                      HiveProject(ss_item_sk=[$2], ss_customer_sk=[$3], ss_store_sk=[$7], ss_ticket_number=[$9], ss_sales_price=[$13])
+                        HiveFilter(condition=[AND(IS NOT NULL($9), IS NOT NULL($2), IS NOT NULL($7), IS NOT NULL($3))])
+                          HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                      HiveJoin(condition=[AND(=($1, $5), <>($4, $8))], joinType=[inner], algorithm=[none], cost=[not available])
+                        HiveProject(c_customer_sk=[$0], c_current_addr_sk=[$4], c_first_name=[$8], c_last_name=[$9], c_birth_country=[$14])
+                          HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($4))])
+                            HiveTableScan(table=[[default, customer]], table:alias=[customer])
+                        HiveJoin(condition=[=($7, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+                          HiveProject(ca_address_sk=[$0], ca_state=[$8], ca_zip=[$9], UPPER=[UPPER($10)])
+                            HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($9))])
+                              HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
+                          HiveProject(s_store_sk=[$0], s_store_name=[$5], s_state=[$24], s_zip=[$25])
+                            HiveFilter(condition=[AND(=($10, 7), IS NOT NULL($0), IS NOT NULL($25))])
+                              HiveTableScan(table=[[default, store]], table:alias=[store])
+                    HiveProject(i_item_sk=[$0], i_current_price=[$5], i_size=[$15], i_color=[$17], i_units=[$18], i_manager_id=[$20])
+                      HiveFilter(condition=[IS NOT NULL($0)])
+                        HiveTableScan(table=[[default, item]], table:alias=[item])
+                  HiveProject(sr_item_sk=[$2], sr_ticket_number=[$9])
+                    HiveFilter(condition=[AND(IS NOT NULL($9), IS NOT NULL($2))])
+                      HiveTableScan(table=[[default, store_returns]], table:alias=[store_returns])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/cbo_query30.q.out b/ql/src/test/results/clientpositive/perf/tez/cbo_query30.q.out
index b4f6701..833652a 100644
--- a/ql/src/test/results/clientpositive/perf/tez/cbo_query30.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/cbo_query30.q.out
@@ -81,22 +81,9 @@ HiveSortLimit(sort0=[$0], sort1=[$1], sort2=[$2], sort3=[$3], sort4=[$4], sort5=
             HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
       HiveProject(wr_returning_customer_sk=[$0], ca_state=[$1], $f2=[$2], _o__c0=[$3], ctr_state=[$4])
         HiveJoin(condition=[AND(=($1, $4), >($2, $3))], joinType=[inner], algorithm=[none], cost=[not available])
-          HiveProject(wr_returning_customer_sk=[$1], ca_state=[$0], $f2=[$2])
-            HiveAggregate(group=[{1, 3}], agg#0=[sum($5)])
-              HiveJoin(condition=[=($4, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                HiveProject(ca_address_sk=[$0], ca_state=[$8])
-                  HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($8))])
-                    HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
-                HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available])
-                  HiveProject(wr_returned_date_sk=[$0], wr_returning_customer_sk=[$7], wr_returning_addr_sk=[$10], wr_return_amt=[$15])
-                    HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($10), IS NOT NULL($7))])
-                      HiveTableScan(table=[[default, web_returns]], table:alias=[web_returns])
-                  HiveProject(d_date_sk=[$0])
-                    HiveFilter(condition=[AND(=($6, 2002), IS NOT NULL($0))])
-                      HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-          HiveProject(_o__c0=[*(/($1, $2), 1.2:DECIMAL(2, 1))], ctr_state=[$0])
-            HiveAggregate(group=[{0}], agg#0=[sum($2)], agg#1=[count($2)])
-              HiveProject(ca_state=[$0], wr_returning_customer_sk=[$1], $f2=[$2])
+          HiveProject(wr_returning_customer_sk=[$0], ca_state=[$1], $f2=[$2])
+            HiveFilter(condition=[IS NOT NULL($2)])
+              HiveProject(wr_returning_customer_sk=[$1], ca_state=[$0], $f2=[$2])
                 HiveAggregate(group=[{1, 3}], agg#0=[sum($5)])
                   HiveJoin(condition=[=($4, $0)], joinType=[inner], algorithm=[none], cost=[not available])
                     HiveProject(ca_address_sk=[$0], ca_state=[$8])
@@ -104,9 +91,25 @@ HiveSortLimit(sort0=[$0], sort1=[$1], sort2=[$2], sort3=[$3], sort4=[$4], sort5=
                         HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
                     HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available])
                       HiveProject(wr_returned_date_sk=[$0], wr_returning_customer_sk=[$7], wr_returning_addr_sk=[$10], wr_return_amt=[$15])
-                        HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($10))])
+                        HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($10), IS NOT NULL($7))])
                           HiveTableScan(table=[[default, web_returns]], table:alias=[web_returns])
                       HiveProject(d_date_sk=[$0])
                         HiveFilter(condition=[AND(=($6, 2002), IS NOT NULL($0))])
                           HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+          HiveProject(_o__c0=[*(/($1, $2), 1.2:DECIMAL(2, 1))], ctr_state=[$0])
+            HiveFilter(condition=[AND(IS NOT NULL($1), IS NOT NULL($2))])
+              HiveAggregate(group=[{0}], agg#0=[sum($2)], agg#1=[count($2)])
+                HiveProject(ca_state=[$0], wr_returning_customer_sk=[$1], $f2=[$2])
+                  HiveAggregate(group=[{1, 3}], agg#0=[sum($5)])
+                    HiveJoin(condition=[=($4, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+                      HiveProject(ca_address_sk=[$0], ca_state=[$8])
+                        HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($8))])
+                          HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
+                      HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available])
+                        HiveProject(wr_returned_date_sk=[$0], wr_returning_customer_sk=[$7], wr_returning_addr_sk=[$10], wr_return_amt=[$15])
+                          HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($10))])
+                            HiveTableScan(table=[[default, web_returns]], table:alias=[web_returns])
+                        HiveProject(d_date_sk=[$0])
+                          HiveFilter(condition=[AND(=($6, 2002), IS NOT NULL($0))])
+                            HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/cbo_query32.q.out b/ql/src/test/results/clientpositive/perf/tez/cbo_query32.q.out
index 621c32e..2720c3b 100644
--- a/ql/src/test/results/clientpositive/perf/tez/cbo_query32.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/cbo_query32.q.out
@@ -62,26 +62,26 @@ POSTHOOK: Input: default@item
 POSTHOOK: Output: hdfs://### HDFS PATH ###
 CBO PLAN:
 HiveAggregate(group=[{}], agg#0=[sum($2)])
-  HiveJoin(condition=[AND(>($2, $5), =($6, $1))], joinType=[inner], algorithm=[none], cost=[not available])
-    HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-      HiveProject(cs_sold_date_sk=[$0], cs_item_sk=[$15], cs_ext_discount_amt=[$22])
-        HiveFilter(condition=[AND(IS NOT NULL($15), IS NOT NULL($0))])
-          HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
-      HiveProject(d_date_sk=[$0])
-        HiveFilter(condition=[AND(BETWEEN(false, CAST($2):TIMESTAMP(9), 1998-03-18 00:00:00:TIMESTAMP(9), 1998-06-16 00:00:00:TIMESTAMP(9)), IS NOT NULL($0))])
-          HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-    HiveProject(cs_item_sk=[$0], CAST=[$1], i_item_sk=[$2])
-      HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-        HiveProject(cs_item_sk=[$0], CAST=[CAST(*(1.3:DECIMAL(2, 1), /($1, $2))):DECIMAL(14, 7)])
-          HiveAggregate(group=[{1}], agg#0=[sum($2)], agg#1=[count($2)])
-            HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-              HiveProject(cs_sold_date_sk=[$0], cs_item_sk=[$15], cs_ext_discount_amt=[$22])
-                HiveFilter(condition=[AND(IS NOT NULL($15), IS NOT NULL($0))])
-                  HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
-              HiveProject(d_date_sk=[$0])
-                HiveFilter(condition=[AND(BETWEEN(false, CAST($2):TIMESTAMP(9), 1998-03-18 00:00:00:TIMESTAMP(9), 1998-06-16 00:00:00:TIMESTAMP(9)), IS NOT NULL($0))])
-                  HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-        HiveProject(i_item_sk=[$0])
-          HiveFilter(condition=[AND(=($13, 269), IS NOT NULL($0))])
-            HiveTableScan(table=[[default, item]], table:alias=[item])
+  HiveJoin(condition=[AND(=($5, $4), >($2, $6))], joinType=[inner], algorithm=[none], cost=[not available])
+    HiveJoin(condition=[=($4, $1)], joinType=[inner], algorithm=[none], cost=[not available])
+      HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+        HiveProject(cs_sold_date_sk=[$0], cs_item_sk=[$15], cs_ext_discount_amt=[$22])
+          HiveFilter(condition=[AND(IS NOT NULL($15), IS NOT NULL($0), IS NOT NULL($22))])
+            HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
+        HiveProject(d_date_sk=[$0])
+          HiveFilter(condition=[AND(BETWEEN(false, CAST($2):TIMESTAMP(9), 1998-03-18 00:00:00:TIMESTAMP(9), 1998-06-16 00:00:00:TIMESTAMP(9)), IS NOT NULL($0))])
+            HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+      HiveProject(i_item_sk=[$0])
+        HiveFilter(condition=[AND(=($13, 269), IS NOT NULL($0))])
+          HiveTableScan(table=[[default, item]], table:alias=[item])
+    HiveProject(cs_item_sk=[$0], CAST=[CAST(*(1.3:DECIMAL(2, 1), /($1, $2))):DECIMAL(14, 7)])
+      HiveFilter(condition=[IS NOT NULL(CAST(*(1.3:DECIMAL(2, 1), /($1, $2))):DECIMAL(14, 7))])
+        HiveAggregate(group=[{1}], agg#0=[sum($2)], agg#1=[count($2)])
+          HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+            HiveProject(cs_sold_date_sk=[$0], cs_item_sk=[$15], cs_ext_discount_amt=[$22])
+              HiveFilter(condition=[AND(IS NOT NULL($15), IS NOT NULL($0))])
+                HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
+            HiveProject(d_date_sk=[$0])
+              HiveFilter(condition=[AND(BETWEEN(false, CAST($2):TIMESTAMP(9), 1998-03-18 00:00:00:TIMESTAMP(9), 1998-06-16 00:00:00:TIMESTAMP(9)), IS NOT NULL($0))])
+                HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/cbo_query54.q.out b/ql/src/test/results/clientpositive/perf/tez/cbo_query54.q.out
index fcb3abe..26f5014 100644
--- a/ql/src/test/results/clientpositive/perf/tez/cbo_query54.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/cbo_query54.q.out
@@ -146,7 +146,7 @@ HiveSortLimit(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC], fetch=[100])
                         HiveFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($0))])
                           HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
                       HiveProject(d_date_sk=[$0], d_month_seq=[$3])
-                        HiveFilter(condition=[IS NOT NULL($0)])
+                        HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($3))])
                           HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
                     HiveProject(ca_address_sk=[$0], ca_county=[$1], ca_state=[$2], s_county=[$3], s_state=[$4], c_customer_sk=[$5], c_current_addr_sk=[$6])
                       HiveJoin(condition=[=($6, $0)], joinType=[inner], algorithm=[none], cost=[not available])
@@ -200,11 +200,11 @@ HiveSortLimit(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC], fetch=[100])
               HiveProject($f0=[$0])
                 HiveAggregate(group=[{0}])
                   HiveProject($f0=[+($3, 1)])
-                    HiveFilter(condition=[AND(=($6, 1999), =($8, 3))])
+                    HiveFilter(condition=[AND(=($6, 1999), =($8, 3), IS NOT NULL($3))])
                       HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
             HiveProject($f0=[$0])
               HiveAggregate(group=[{0}])
                 HiveProject($f0=[+($3, 3)])
-                  HiveFilter(condition=[AND(=($6, 1999), =($8, 3))])
+                  HiveFilter(condition=[AND(=($6, 1999), =($8, 3), IS NOT NULL($3))])
                     HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/cbo_query6.q.out b/ql/src/test/results/clientpositive/perf/tez/cbo_query6.q.out
index b8fd5a1..4c4b76f 100644
--- a/ql/src/test/results/clientpositive/perf/tez/cbo_query6.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/cbo_query6.q.out
@@ -1,4 +1,4 @@
-Warning: Map Join MAPJOIN[171][bigTable=?] in task 'Reducer 15' is a cross product
+Warning: Map Join MAPJOIN[172][bigTable=?] in task 'Reducer 15' is a cross product
 PREHOOK: query: explain cbo
 select  a.ca_state state, count(*) cnt
  from customer_address a
@@ -91,13 +91,14 @@ HiveSortLimit(sort0=[$1], dir0=[ASC], fetch=[100])
           HiveProject(i_item_sk=[$0], i_current_price=[$1], i_category=[$2], i_category0=[$3], *=[$4], cnt=[$5])
             HiveJoin(condition=[AND(=($3, $2), >($1, $4))], joinType=[inner], algorithm=[none], cost=[not available])
               HiveProject(i_item_sk=[$0], i_current_price=[$5], i_category=[$12])
-                HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($12))])
+                HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($12), IS NOT NULL($5))])
                   HiveTableScan(table=[[default, item]], table:alias=[i])
               HiveJoin(condition=[true], joinType=[inner], algorithm=[none], cost=[not available])
                 HiveProject(i_category=[$0], *=[*(1.2:DECIMAL(2, 1), CAST(/($1, $2)):DECIMAL(16, 6))])
-                  HiveAggregate(group=[{12}], agg#0=[sum($5)], agg#1=[count($5)])
-                    HiveFilter(condition=[IS NOT NULL($12)])
-                      HiveTableScan(table=[[default, item]], table:alias=[j])
+                  HiveFilter(condition=[IS NOT NULL(CAST(/($1, $2)):DECIMAL(16, 6))])
+                    HiveAggregate(group=[{12}], agg#0=[sum($5)], agg#1=[count($5)])
+                      HiveFilter(condition=[IS NOT NULL($12)])
+                        HiveTableScan(table=[[default, item]], table:alias=[j])
                 HiveProject(cnt=[$0])
                   HiveFilter(condition=[<=(sq_count_check($0), 1)])
                     HiveProject(cnt=[$0])
diff --git a/ql/src/test/results/clientpositive/perf/tez/cbo_query64.q.out b/ql/src/test/results/clientpositive/perf/tez/cbo_query64.q.out
index 3e1cca4..6a3247f 100644
--- a/ql/src/test/results/clientpositive/perf/tez/cbo_query64.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/cbo_query64.q.out
@@ -267,168 +267,172 @@ HiveProject(product_name=[$0], store_name=[$1], store_zip=[$2], b_street_number=
   HiveSortLimit(sort0=[$0], sort1=[$1], sort2=[$18], dir0=[ASC], dir1=[ASC], dir2=[ASC])
     HiveProject(product_name=[$0], store_name=[$2], store_zip=[$3], b_street_number=[$4], b_streen_name=[$5], b_city=[$6], b_zip=[$7], c_street_number=[$8], c_street_name=[$9], c_city=[$10], c_zip=[$11], cnt=[$12], s1=[$13], s2=[$14], s3=[$15], s11=[$20], s21=[$21], s31=[$22], cnt1=[$19])
       HiveJoin(condition=[AND(=($1, $16), <=($19, $12), =($2, $17), =($3, $18))], joinType=[inner], algorithm=[none], cost=[not available])
-        HiveProject($f0=[$13], $f1=[$12], $f2=[$10], $f3=[$11], $f4=[$6], $f5=[$7], $f6=[$8], $f7=[$9], $f8=[$2], $f9=[$3], $f10=[$4], $f11=[$5], $f15=[$14], $f16=[$15], $f17=[$16], $f18=[$17])
-          HiveAggregate(group=[{9, 11, 16, 17, 18, 19, 25, 26, 27, 28, 30, 31, 47, 48}], agg#0=[count()], agg#1=[sum($44)], agg#2=[sum($45)], agg#3=[sum($46)])
-            HiveJoin(condition=[=($33, $51)], joinType=[inner], algorithm=[none], cost=[not available])
-              HiveJoin(condition=[AND(<>($1, $21), =($38, $0))], joinType=[inner], algorithm=[none], cost=[not available])
-                HiveProject(cd_demo_sk=[$0], cd_marital_status=[$2])
-                  HiveFilter(condition=[IS NOT NULL($0)])
-                    HiveTableScan(table=[[default, customer_demographics]], table:alias=[cd1])
-                HiveJoin(condition=[=($35, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                  HiveJoin(condition=[=($1, $18)], joinType=[inner], algorithm=[none], cost=[not available])
-                    HiveJoin(condition=[=($3, $13)], joinType=[inner], algorithm=[none], cost=[not available])
-                      HiveJoin(condition=[=($2, $10)], joinType=[inner], algorithm=[none], cost=[not available])
-                        HiveJoin(condition=[=($4, $8)], joinType=[inner], algorithm=[none], cost=[not available])
-                          HiveJoin(condition=[=($5, $6)], joinType=[inner], algorithm=[none], cost=[not available])
-                            HiveProject(c_customer_sk=[$0], c_current_cdemo_sk=[$2], c_current_hdemo_sk=[$3], c_current_addr_sk=[$4], c_first_shipto_date_sk=[$5], c_first_sales_date_sk=[$6])
-                              HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($6), IS NOT NULL($5), IS NOT NULL($2), IS NOT NULL($3), IS NOT NULL($4))])
-                                HiveTableScan(table=[[default, customer]], table:alias=[customer])
-                            HiveProject(d_date_sk=[$0], d_year=[$6])
-                              HiveFilter(condition=[IS NOT NULL($0)])
-                                HiveTableScan(table=[[default, date_dim]], table:alias=[d2])
-                          HiveProject(d_date_sk=[$0], d_year=[$6])
-                            HiveFilter(condition=[IS NOT NULL($0)])
-                              HiveTableScan(table=[[default, date_dim]], table:alias=[d3])
-                        HiveProject(hd_demo_sk=[$0], hd_income_band_sk=[$1], ib_income_band_sk=[$2])
-                          HiveJoin(condition=[=($1, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-                            HiveProject(hd_demo_sk=[$0], hd_income_band_sk=[$1])
-                              HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($1))])
-                                HiveTableScan(table=[[default, household_demographics]], table:alias=[hd2])
-                            HiveProject(ib_income_band_sk=[$0])
-                              HiveFilter(condition=[IS NOT NULL($0)])
-                                HiveTableScan(table=[[default, income_band]], table:alias=[ib2])
-                      HiveProject(ca_address_sk=[$0], ca_street_number=[$2], ca_street_name=[$3], ca_city=[$6], ca_zip=[$9])
-                        HiveFilter(condition=[IS NOT NULL($0)])
-                          HiveTableScan(table=[[default, customer_address]], table:alias=[ad2])
+        HiveProject($f0=[$0], $f1=[$1], $f2=[$2], $f3=[$3], $f4=[$4], $f5=[$5], $f6=[$6], $f7=[$7], $f8=[$8], $f9=[$9], $f10=[$10], $f11=[$11], $f15=[$14], $f16=[$15], $f17=[$16], $f18=[$17])
+          HiveFilter(condition=[IS NOT NULL($14)])
+            HiveProject(i_product_name=[$13], i_item_sk=[$12], s_store_name=[$10], s_zip=[$11], ca_street_number=[$6], ca_street_name=[$7], ca_city=[$8], ca_zip=[$9], ca_street_number0=[$2], ca_street_name0=[$3], ca_city0=[$4], ca_zip0=[$5], d_year=[$0], d_year0=[$1], $f14=[$14], $f15=[$15], $f16=[$16], $f17=[$17])
+              HiveAggregate(group=[{9, 11, 16, 17, 18, 19, 25, 26, 27, 28, 30, 31, 47, 48}], agg#0=[count()], agg#1=[sum($44)], agg#2=[sum($45)], agg#3=[sum($46)])
+                HiveJoin(condition=[=($33, $51)], joinType=[inner], algorithm=[none], cost=[not available])
+                  HiveJoin(condition=[AND(<>($1, $21), =($38, $0))], joinType=[inner], algorithm=[none], cost=[not available])
                     HiveProject(cd_demo_sk=[$0], cd_marital_status=[$2])
                       HiveFilter(condition=[IS NOT NULL($0)])
-                        HiveTableScan(table=[[default, customer_demographics]], table:alias=[cd2])
-                  HiveProject(sr_item_sk=[$0], sr_ticket_number=[$1], ca_address_sk=[$2], ca_street_number=[$3], ca_street_name=[$4], ca_city=[$5], ca_zip=[$6], s_store_sk=[$7], s_store_name=[$8], s_zip=[$9], hd_demo_sk=[$10], hd_income_band_sk=[$11], p_promo_sk=[$12], ss_sold_date_sk=[$13], ss_item_sk=[$14], ss_customer_sk=[$15], ss_cdemo_sk=[$16], ss_hdemo_sk=[$17], ss_addr_sk=[$18], ss_store_sk=[$19], ss_promo_sk=[$20], ss_ticket_number=[$21], ss_wholesale_cost=[$22], ss_list_price=[$ [...]
-                    HiveJoin(condition=[AND(=($14, $0), =($21, $1))], joinType=[inner], algorithm=[none], cost=[not available])
-                      HiveProject(sr_item_sk=[$2], sr_ticket_number=[$9])
-                        HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($9))])
-                          HiveTableScan(table=[[default, store_returns]], table:alias=[store_returns])
-                      HiveJoin(condition=[=($16, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                        HiveProject(ca_address_sk=[$0], ca_street_number=[$2], ca_street_name=[$3], ca_city=[$6], ca_zip=[$9])
-                          HiveFilter(condition=[IS NOT NULL($0)])
-                            HiveTableScan(table=[[default, customer_address]], table:alias=[ad1])
-                        HiveJoin(condition=[=($12, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                          HiveProject(s_store_sk=[$0], s_store_name=[$5], s_zip=[$25])
-                            HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($5), IS NOT NULL($25))])
-                              HiveTableScan(table=[[default, store]], table:alias=[store])
-                          HiveJoin(condition=[=($7, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                            HiveProject(hd_demo_sk=[$0], hd_income_band_sk=[$1])
-                              HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($1))])
-                                HiveTableScan(table=[[default, household_demographics]], table:alias=[hd1])
-                            HiveJoin(condition=[=($8, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                              HiveProject(p_promo_sk=[$0])
+                        HiveTableScan(table=[[default, customer_demographics]], table:alias=[cd1])
+                    HiveJoin(condition=[=($35, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+                      HiveJoin(condition=[=($1, $18)], joinType=[inner], algorithm=[none], cost=[not available])
+                        HiveJoin(condition=[=($3, $13)], joinType=[inner], algorithm=[none], cost=[not available])
+                          HiveJoin(condition=[=($2, $10)], joinType=[inner], algorithm=[none], cost=[not available])
+                            HiveJoin(condition=[=($4, $8)], joinType=[inner], algorithm=[none], cost=[not available])
+                              HiveJoin(condition=[=($5, $6)], joinType=[inner], algorithm=[none], cost=[not available])
+                                HiveProject(c_customer_sk=[$0], c_current_cdemo_sk=[$2], c_current_hdemo_sk=[$3], c_current_addr_sk=[$4], c_first_shipto_date_sk=[$5], c_first_sales_date_sk=[$6])
+                                  HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($6), IS NOT NULL($5), IS NOT NULL($2), IS NOT NULL($3), IS NOT NULL($4))])
+                                    HiveTableScan(table=[[default, customer]], table:alias=[customer])
+                                HiveProject(d_date_sk=[$0], d_year=[$6])
+                                  HiveFilter(condition=[IS NOT NULL($0)])
+                                    HiveTableScan(table=[[default, date_dim]], table:alias=[d2])
+                              HiveProject(d_date_sk=[$0], d_year=[$6])
                                 HiveFilter(condition=[IS NOT NULL($0)])
-                                  HiveTableScan(table=[[default, promotion]], table:alias=[promotion])
-                              HiveJoin(condition=[=($1, $15)], joinType=[inner], algorithm=[none], cost=[not available])
-                                HiveJoin(condition=[=($0, $14)], joinType=[inner], algorithm=[none], cost=[not available])
-                                  HiveJoin(condition=[=($1, $12)], joinType=[inner], algorithm=[none], cost=[not available])
-                                    HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2], ss_customer_sk=[$3], ss_cdemo_sk=[$4], ss_hdemo_sk=[$5], ss_addr_sk=[$6], ss_store_sk=[$7], ss_promo_sk=[$8], ss_ticket_number=[$9], ss_wholesale_cost=[$11], ss_list_price=[$12], ss_coupon_amt=[$19])
-                                      HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($9), IS NOT NULL($0), IS NOT NULL($7), IS NOT NULL($3), IS NOT NULL($4), IS NOT NULL($8), IS NOT NULL($5), IS NOT NULL($6))])
-                                        HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                                    HiveProject(i_item_sk=[$0], i_product_name=[$21])
-                                      HiveFilter(condition=[AND(IN($17, _UTF-16LE'maroon', _UTF-16LE'burnished', _UTF-16LE'dim', _UTF-16LE'steel', _UTF-16LE'navajo', _UTF-16LE'chocolate'), BETWEEN(false, $5, 36:DECIMAL(2, 0), 45:DECIMAL(2, 0)), IS NOT NULL($0))])
-                                        HiveTableScan(table=[[default, item]], table:alias=[item])
-                                  HiveProject(d_date_sk=[$0])
-                                    HiveFilter(condition=[AND(=($6, 2000), IS NOT NULL($0))])
-                                      HiveTableScan(table=[[default, date_dim]], table:alias=[d1])
-                                HiveProject($f0=[$0])
-                                  HiveFilter(condition=[>($1, *(2:DECIMAL(10, 0), $2))])
-                                    HiveAggregate(group=[{0}], agg#0=[sum($2)], agg#1=[sum($5)])
-                                      HiveJoin(condition=[AND(=($0, $3), =($1, $4))], joinType=[inner], algorithm=[none], cost=[not available])
-                                        HiveProject(cs_item_sk=[$15], cs_order_number=[$17], cs_ext_list_price=[$25])
-                                          HiveFilter(condition=[AND(IS NOT NULL($15), IS NOT NULL($17))])
-                                            HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
-                                        HiveProject(cr_item_sk=[$2], cr_order_number=[$16], +=[+(+($23, $24), $25)])
-                                          HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($16))])
-                                            HiveTableScan(table=[[default, catalog_returns]], table:alias=[catalog_returns])
-              HiveProject(ib_income_band_sk=[$0])
-                HiveFilter(condition=[IS NOT NULL($0)])
-                  HiveTableScan(table=[[default, income_band]], table:alias=[ib1])
-        HiveProject($f1=[$12], $f2=[$10], $f3=[$11], $f15=[$14], $f16=[$15], $f17=[$16], $f18=[$17])
-          HiveAggregate(group=[{9, 11, 16, 17, 18, 19, 25, 26, 27, 28, 30, 31, 47, 48}], agg#0=[count()], agg#1=[sum($44)], agg#2=[sum($45)], agg#3=[sum($46)])
-            HiveJoin(condition=[=($33, $51)], joinType=[inner], algorithm=[none], cost=[not available])
-              HiveJoin(condition=[AND(<>($1, $21), =($38, $0))], joinType=[inner], algorithm=[none], cost=[not available])
-                HiveProject(cd_demo_sk=[$0], cd_marital_status=[$2])
-                  HiveFilter(condition=[IS NOT NULL($0)])
-                    HiveTableScan(table=[[default, customer_demographics]], table:alias=[cd1])
-                HiveJoin(condition=[=($35, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                  HiveJoin(condition=[=($1, $18)], joinType=[inner], algorithm=[none], cost=[not available])
-                    HiveJoin(condition=[=($3, $13)], joinType=[inner], algorithm=[none], cost=[not available])
-                      HiveJoin(condition=[=($2, $10)], joinType=[inner], algorithm=[none], cost=[not available])
-                        HiveJoin(condition=[=($4, $8)], joinType=[inner], algorithm=[none], cost=[not available])
-                          HiveJoin(condition=[=($5, $6)], joinType=[inner], algorithm=[none], cost=[not available])
-                            HiveProject(c_customer_sk=[$0], c_current_cdemo_sk=[$2], c_current_hdemo_sk=[$3], c_current_addr_sk=[$4], c_first_shipto_date_sk=[$5], c_first_sales_date_sk=[$6])
-                              HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($6), IS NOT NULL($5), IS NOT NULL($2), IS NOT NULL($3), IS NOT NULL($4))])
-                                HiveTableScan(table=[[default, customer]], table:alias=[customer])
-                            HiveProject(d_date_sk=[$0], d_year=[$6])
-                              HiveFilter(condition=[IS NOT NULL($0)])
-                                HiveTableScan(table=[[default, date_dim]], table:alias=[d2])
-                          HiveProject(d_date_sk=[$0], d_year=[$6])
+                                  HiveTableScan(table=[[default, date_dim]], table:alias=[d3])
+                            HiveProject(hd_demo_sk=[$0], hd_income_band_sk=[$1], ib_income_band_sk=[$2])
+                              HiveJoin(condition=[=($1, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+                                HiveProject(hd_demo_sk=[$0], hd_income_band_sk=[$1])
+                                  HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($1))])
+                                    HiveTableScan(table=[[default, household_demographics]], table:alias=[hd2])
+                                HiveProject(ib_income_band_sk=[$0])
+                                  HiveFilter(condition=[IS NOT NULL($0)])
+                                    HiveTableScan(table=[[default, income_band]], table:alias=[ib2])
+                          HiveProject(ca_address_sk=[$0], ca_street_number=[$2], ca_street_name=[$3], ca_city=[$6], ca_zip=[$9])
                             HiveFilter(condition=[IS NOT NULL($0)])
-                              HiveTableScan(table=[[default, date_dim]], table:alias=[d3])
-                        HiveProject(hd_demo_sk=[$0], hd_income_band_sk=[$1], ib_income_band_sk=[$2])
-                          HiveJoin(condition=[=($1, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-                            HiveProject(hd_demo_sk=[$0], hd_income_band_sk=[$1])
-                              HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($1))])
-                                HiveTableScan(table=[[default, household_demographics]], table:alias=[hd2])
-                            HiveProject(ib_income_band_sk=[$0])
+                              HiveTableScan(table=[[default, customer_address]], table:alias=[ad2])
+                        HiveProject(cd_demo_sk=[$0], cd_marital_status=[$2])
+                          HiveFilter(condition=[IS NOT NULL($0)])
+                            HiveTableScan(table=[[default, customer_demographics]], table:alias=[cd2])
+                      HiveProject(sr_item_sk=[$0], sr_ticket_number=[$1], ca_address_sk=[$2], ca_street_number=[$3], ca_street_name=[$4], ca_city=[$5], ca_zip=[$6], s_store_sk=[$7], s_store_name=[$8], s_zip=[$9], hd_demo_sk=[$10], hd_income_band_sk=[$11], p_promo_sk=[$12], ss_sold_date_sk=[$13], ss_item_sk=[$14], ss_customer_sk=[$15], ss_cdemo_sk=[$16], ss_hdemo_sk=[$17], ss_addr_sk=[$18], ss_store_sk=[$19], ss_promo_sk=[$20], ss_ticket_number=[$21], ss_wholesale_cost=[$22], ss_list_pric [...]
+                        HiveJoin(condition=[AND(=($14, $0), =($21, $1))], joinType=[inner], algorithm=[none], cost=[not available])
+                          HiveProject(sr_item_sk=[$2], sr_ticket_number=[$9])
+                            HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($9))])
+                              HiveTableScan(table=[[default, store_returns]], table:alias=[store_returns])
+                          HiveJoin(condition=[=($16, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+                            HiveProject(ca_address_sk=[$0], ca_street_number=[$2], ca_street_name=[$3], ca_city=[$6], ca_zip=[$9])
                               HiveFilter(condition=[IS NOT NULL($0)])
-                                HiveTableScan(table=[[default, income_band]], table:alias=[ib2])
-                      HiveProject(ca_address_sk=[$0], ca_street_number=[$2], ca_street_name=[$3], ca_city=[$6], ca_zip=[$9])
-                        HiveFilter(condition=[IS NOT NULL($0)])
-                          HiveTableScan(table=[[default, customer_address]], table:alias=[ad2])
+                                HiveTableScan(table=[[default, customer_address]], table:alias=[ad1])
+                            HiveJoin(condition=[=($12, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+                              HiveProject(s_store_sk=[$0], s_store_name=[$5], s_zip=[$25])
+                                HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($5), IS NOT NULL($25))])
+                                  HiveTableScan(table=[[default, store]], table:alias=[store])
+                              HiveJoin(condition=[=($7, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+                                HiveProject(hd_demo_sk=[$0], hd_income_band_sk=[$1])
+                                  HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($1))])
+                                    HiveTableScan(table=[[default, household_demographics]], table:alias=[hd1])
+                                HiveJoin(condition=[=($8, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+                                  HiveProject(p_promo_sk=[$0])
+                                    HiveFilter(condition=[IS NOT NULL($0)])
+                                      HiveTableScan(table=[[default, promotion]], table:alias=[promotion])
+                                  HiveJoin(condition=[=($1, $15)], joinType=[inner], algorithm=[none], cost=[not available])
+                                    HiveJoin(condition=[=($0, $14)], joinType=[inner], algorithm=[none], cost=[not available])
+                                      HiveJoin(condition=[=($1, $12)], joinType=[inner], algorithm=[none], cost=[not available])
+                                        HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2], ss_customer_sk=[$3], ss_cdemo_sk=[$4], ss_hdemo_sk=[$5], ss_addr_sk=[$6], ss_store_sk=[$7], ss_promo_sk=[$8], ss_ticket_number=[$9], ss_wholesale_cost=[$11], ss_list_price=[$12], ss_coupon_amt=[$19])
+                                          HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($9), IS NOT NULL($0), IS NOT NULL($7), IS NOT NULL($3), IS NOT NULL($4), IS NOT NULL($8), IS NOT NULL($5), IS NOT NULL($6))])
+                                            HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                                        HiveProject(i_item_sk=[$0], i_product_name=[$21])
+                                          HiveFilter(condition=[AND(IN($17, _UTF-16LE'maroon', _UTF-16LE'burnished', _UTF-16LE'dim', _UTF-16LE'steel', _UTF-16LE'navajo', _UTF-16LE'chocolate'), BETWEEN(false, $5, 36:DECIMAL(2, 0), 45:DECIMAL(2, 0)), IS NOT NULL($0))])
+                                            HiveTableScan(table=[[default, item]], table:alias=[item])
+                                      HiveProject(d_date_sk=[$0])
+                                        HiveFilter(condition=[AND(=($6, 2000), IS NOT NULL($0))])
+                                          HiveTableScan(table=[[default, date_dim]], table:alias=[d1])
+                                    HiveProject($f0=[$0])
+                                      HiveFilter(condition=[>($1, *(2:DECIMAL(10, 0), $2))])
+                                        HiveAggregate(group=[{0}], agg#0=[sum($2)], agg#1=[sum($5)])
+                                          HiveJoin(condition=[AND(=($0, $3), =($1, $4))], joinType=[inner], algorithm=[none], cost=[not available])
+                                            HiveProject(cs_item_sk=[$15], cs_order_number=[$17], cs_ext_list_price=[$25])
+                                              HiveFilter(condition=[AND(IS NOT NULL($15), IS NOT NULL($17))])
+                                                HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
+                                            HiveProject(cr_item_sk=[$2], cr_order_number=[$16], +=[+(+($23, $24), $25)])
+                                              HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($16))])
+                                                HiveTableScan(table=[[default, catalog_returns]], table:alias=[catalog_returns])
+                  HiveProject(ib_income_band_sk=[$0])
+                    HiveFilter(condition=[IS NOT NULL($0)])
+                      HiveTableScan(table=[[default, income_band]], table:alias=[ib1])
+        HiveProject($f1=[$1], $f2=[$2], $f3=[$3], $f15=[$14], $f16=[$15], $f17=[$16], $f18=[$17])
+          HiveFilter(condition=[IS NOT NULL($14)])
+            HiveProject(i_product_name=[$13], i_item_sk=[$12], s_store_name=[$10], s_zip=[$11], ca_street_number=[$6], ca_street_name=[$7], ca_city=[$8], ca_zip=[$9], ca_street_number0=[$2], ca_street_name0=[$3], ca_city0=[$4], ca_zip0=[$5], d_year=[$0], d_year0=[$1], $f14=[$14], $f15=[$15], $f16=[$16], $f17=[$17])
+              HiveAggregate(group=[{9, 11, 16, 17, 18, 19, 25, 26, 27, 28, 30, 31, 47, 48}], agg#0=[count()], agg#1=[sum($44)], agg#2=[sum($45)], agg#3=[sum($46)])
+                HiveJoin(condition=[=($33, $51)], joinType=[inner], algorithm=[none], cost=[not available])
+                  HiveJoin(condition=[AND(<>($1, $21), =($38, $0))], joinType=[inner], algorithm=[none], cost=[not available])
                     HiveProject(cd_demo_sk=[$0], cd_marital_status=[$2])
                       HiveFilter(condition=[IS NOT NULL($0)])
-                        HiveTableScan(table=[[default, customer_demographics]], table:alias=[cd2])
-                  HiveProject(sr_item_sk=[$0], sr_ticket_number=[$1], ca_address_sk=[$2], ca_street_number=[$3], ca_street_name=[$4], ca_city=[$5], ca_zip=[$6], s_store_sk=[$7], s_store_name=[$8], s_zip=[$9], hd_demo_sk=[$10], hd_income_band_sk=[$11], p_promo_sk=[$12], ss_sold_date_sk=[$13], ss_item_sk=[$14], ss_customer_sk=[$15], ss_cdemo_sk=[$16], ss_hdemo_sk=[$17], ss_addr_sk=[$18], ss_store_sk=[$19], ss_promo_sk=[$20], ss_ticket_number=[$21], ss_wholesale_cost=[$22], ss_list_price=[$ [...]
-                    HiveJoin(condition=[AND(=($14, $0), =($21, $1))], joinType=[inner], algorithm=[none], cost=[not available])
-                      HiveProject(sr_item_sk=[$2], sr_ticket_number=[$9])
-                        HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($9))])
-                          HiveTableScan(table=[[default, store_returns]], table:alias=[store_returns])
-                      HiveJoin(condition=[=($16, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                        HiveProject(ca_address_sk=[$0], ca_street_number=[$2], ca_street_name=[$3], ca_city=[$6], ca_zip=[$9])
-                          HiveFilter(condition=[IS NOT NULL($0)])
-                            HiveTableScan(table=[[default, customer_address]], table:alias=[ad1])
-                        HiveJoin(condition=[=($12, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                          HiveProject(s_store_sk=[$0], s_store_name=[$5], s_zip=[$25])
-                            HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($5), IS NOT NULL($25))])
-                              HiveTableScan(table=[[default, store]], table:alias=[store])
-                          HiveJoin(condition=[=($7, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                            HiveProject(hd_demo_sk=[$0], hd_income_band_sk=[$1])
-                              HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($1))])
-                                HiveTableScan(table=[[default, household_demographics]], table:alias=[hd1])
-                            HiveJoin(condition=[=($8, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                              HiveProject(p_promo_sk=[$0])
+                        HiveTableScan(table=[[default, customer_demographics]], table:alias=[cd1])
+                    HiveJoin(condition=[=($35, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+                      HiveJoin(condition=[=($1, $18)], joinType=[inner], algorithm=[none], cost=[not available])
+                        HiveJoin(condition=[=($3, $13)], joinType=[inner], algorithm=[none], cost=[not available])
+                          HiveJoin(condition=[=($2, $10)], joinType=[inner], algorithm=[none], cost=[not available])
+                            HiveJoin(condition=[=($4, $8)], joinType=[inner], algorithm=[none], cost=[not available])
+                              HiveJoin(condition=[=($5, $6)], joinType=[inner], algorithm=[none], cost=[not available])
+                                HiveProject(c_customer_sk=[$0], c_current_cdemo_sk=[$2], c_current_hdemo_sk=[$3], c_current_addr_sk=[$4], c_first_shipto_date_sk=[$5], c_first_sales_date_sk=[$6])
+                                  HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($6), IS NOT NULL($5), IS NOT NULL($2), IS NOT NULL($3), IS NOT NULL($4))])
+                                    HiveTableScan(table=[[default, customer]], table:alias=[customer])
+                                HiveProject(d_date_sk=[$0], d_year=[$6])
+                                  HiveFilter(condition=[IS NOT NULL($0)])
+                                    HiveTableScan(table=[[default, date_dim]], table:alias=[d2])
+                              HiveProject(d_date_sk=[$0], d_year=[$6])
                                 HiveFilter(condition=[IS NOT NULL($0)])
-                                  HiveTableScan(table=[[default, promotion]], table:alias=[promotion])
-                              HiveJoin(condition=[=($1, $15)], joinType=[inner], algorithm=[none], cost=[not available])
-                                HiveJoin(condition=[=($0, $14)], joinType=[inner], algorithm=[none], cost=[not available])
-                                  HiveJoin(condition=[=($1, $12)], joinType=[inner], algorithm=[none], cost=[not available])
-                                    HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2], ss_customer_sk=[$3], ss_cdemo_sk=[$4], ss_hdemo_sk=[$5], ss_addr_sk=[$6], ss_store_sk=[$7], ss_promo_sk=[$8], ss_ticket_number=[$9], ss_wholesale_cost=[$11], ss_list_price=[$12], ss_coupon_amt=[$19])
-                                      HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($9), IS NOT NULL($0), IS NOT NULL($7), IS NOT NULL($3), IS NOT NULL($4), IS NOT NULL($8), IS NOT NULL($5), IS NOT NULL($6))])
-                                        HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                                    HiveProject(i_item_sk=[$0], i_product_name=[$21])
-                                      HiveFilter(condition=[AND(IN($17, _UTF-16LE'maroon', _UTF-16LE'burnished', _UTF-16LE'dim', _UTF-16LE'steel', _UTF-16LE'navajo', _UTF-16LE'chocolate'), BETWEEN(false, $5, 36:DECIMAL(2, 0), 45:DECIMAL(2, 0)), IS NOT NULL($0))])
-                                        HiveTableScan(table=[[default, item]], table:alias=[item])
-                                  HiveProject(d_date_sk=[$0])
-                                    HiveFilter(condition=[AND(=($6, 2001), IS NOT NULL($0))])
-                                      HiveTableScan(table=[[default, date_dim]], table:alias=[d1])
-                                HiveProject($f0=[$0])
-                                  HiveFilter(condition=[>($1, *(2:DECIMAL(10, 0), $2))])
-                                    HiveAggregate(group=[{0}], agg#0=[sum($2)], agg#1=[sum($5)])
-                                      HiveJoin(condition=[AND(=($0, $3), =($1, $4))], joinType=[inner], algorithm=[none], cost=[not available])
-                                        HiveProject(cs_item_sk=[$15], cs_order_number=[$17], cs_ext_list_price=[$25])
-                                          HiveFilter(condition=[AND(IS NOT NULL($15), IS NOT NULL($17))])
-                                            HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
-                                        HiveProject(cr_item_sk=[$2], cr_order_number=[$16], +=[+(+($23, $24), $25)])
-                                          HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($16))])
-                                            HiveTableScan(table=[[default, catalog_returns]], table:alias=[catalog_returns])
-              HiveProject(ib_income_band_sk=[$0])
-                HiveFilter(condition=[IS NOT NULL($0)])
-                  HiveTableScan(table=[[default, income_band]], table:alias=[ib1])
+                                  HiveTableScan(table=[[default, date_dim]], table:alias=[d3])
+                            HiveProject(hd_demo_sk=[$0], hd_income_band_sk=[$1], ib_income_band_sk=[$2])
+                              HiveJoin(condition=[=($1, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+                                HiveProject(hd_demo_sk=[$0], hd_income_band_sk=[$1])
+                                  HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($1))])
+                                    HiveTableScan(table=[[default, household_demographics]], table:alias=[hd2])
+                                HiveProject(ib_income_band_sk=[$0])
+                                  HiveFilter(condition=[IS NOT NULL($0)])
+                                    HiveTableScan(table=[[default, income_band]], table:alias=[ib2])
+                          HiveProject(ca_address_sk=[$0], ca_street_number=[$2], ca_street_name=[$3], ca_city=[$6], ca_zip=[$9])
+                            HiveFilter(condition=[IS NOT NULL($0)])
+                              HiveTableScan(table=[[default, customer_address]], table:alias=[ad2])
+                        HiveProject(cd_demo_sk=[$0], cd_marital_status=[$2])
+                          HiveFilter(condition=[IS NOT NULL($0)])
+                            HiveTableScan(table=[[default, customer_demographics]], table:alias=[cd2])
+                      HiveProject(sr_item_sk=[$0], sr_ticket_number=[$1], ca_address_sk=[$2], ca_street_number=[$3], ca_street_name=[$4], ca_city=[$5], ca_zip=[$6], s_store_sk=[$7], s_store_name=[$8], s_zip=[$9], hd_demo_sk=[$10], hd_income_band_sk=[$11], p_promo_sk=[$12], ss_sold_date_sk=[$13], ss_item_sk=[$14], ss_customer_sk=[$15], ss_cdemo_sk=[$16], ss_hdemo_sk=[$17], ss_addr_sk=[$18], ss_store_sk=[$19], ss_promo_sk=[$20], ss_ticket_number=[$21], ss_wholesale_cost=[$22], ss_list_pric [...]
+                        HiveJoin(condition=[AND(=($14, $0), =($21, $1))], joinType=[inner], algorithm=[none], cost=[not available])
+                          HiveProject(sr_item_sk=[$2], sr_ticket_number=[$9])
+                            HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($9))])
+                              HiveTableScan(table=[[default, store_returns]], table:alias=[store_returns])
+                          HiveJoin(condition=[=($16, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+                            HiveProject(ca_address_sk=[$0], ca_street_number=[$2], ca_street_name=[$3], ca_city=[$6], ca_zip=[$9])
+                              HiveFilter(condition=[IS NOT NULL($0)])
+                                HiveTableScan(table=[[default, customer_address]], table:alias=[ad1])
+                            HiveJoin(condition=[=($12, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+                              HiveProject(s_store_sk=[$0], s_store_name=[$5], s_zip=[$25])
+                                HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($5), IS NOT NULL($25))])
+                                  HiveTableScan(table=[[default, store]], table:alias=[store])
+                              HiveJoin(condition=[=($7, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+                                HiveProject(hd_demo_sk=[$0], hd_income_band_sk=[$1])
+                                  HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($1))])
+                                    HiveTableScan(table=[[default, household_demographics]], table:alias=[hd1])
+                                HiveJoin(condition=[=($8, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+                                  HiveProject(p_promo_sk=[$0])
+                                    HiveFilter(condition=[IS NOT NULL($0)])
+                                      HiveTableScan(table=[[default, promotion]], table:alias=[promotion])
+                                  HiveJoin(condition=[=($1, $15)], joinType=[inner], algorithm=[none], cost=[not available])
+                                    HiveJoin(condition=[=($0, $14)], joinType=[inner], algorithm=[none], cost=[not available])
+                                      HiveJoin(condition=[=($1, $12)], joinType=[inner], algorithm=[none], cost=[not available])
+                                        HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2], ss_customer_sk=[$3], ss_cdemo_sk=[$4], ss_hdemo_sk=[$5], ss_addr_sk=[$6], ss_store_sk=[$7], ss_promo_sk=[$8], ss_ticket_number=[$9], ss_wholesale_cost=[$11], ss_list_price=[$12], ss_coupon_amt=[$19])
+                                          HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($9), IS NOT NULL($0), IS NOT NULL($7), IS NOT NULL($3), IS NOT NULL($4), IS NOT NULL($8), IS NOT NULL($5), IS NOT NULL($6))])
+                                            HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                                        HiveProject(i_item_sk=[$0], i_product_name=[$21])
+                                          HiveFilter(condition=[AND(IN($17, _UTF-16LE'maroon', _UTF-16LE'burnished', _UTF-16LE'dim', _UTF-16LE'steel', _UTF-16LE'navajo', _UTF-16LE'chocolate'), BETWEEN(false, $5, 36:DECIMAL(2, 0), 45:DECIMAL(2, 0)), IS NOT NULL($0))])
+                                            HiveTableScan(table=[[default, item]], table:alias=[item])
+                                      HiveProject(d_date_sk=[$0])
+                                        HiveFilter(condition=[AND(=($6, 2001), IS NOT NULL($0))])
+                                          HiveTableScan(table=[[default, date_dim]], table:alias=[d1])
+                                    HiveProject($f0=[$0])
+                                      HiveFilter(condition=[>($1, *(2:DECIMAL(10, 0), $2))])
+                                        HiveAggregate(group=[{0}], agg#0=[sum($2)], agg#1=[sum($5)])
+                                          HiveJoin(condition=[AND(=($0, $3), =($1, $4))], joinType=[inner], algorithm=[none], cost=[not available])
+                                            HiveProject(cs_item_sk=[$15], cs_order_number=[$17], cs_ext_list_price=[$25])
+                                              HiveFilter(condition=[AND(IS NOT NULL($15), IS NOT NULL($17))])
+                                                HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
+                                            HiveProject(cr_item_sk=[$2], cr_order_number=[$16], +=[+(+($23, $24), $25)])
+                                              HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($16))])
+                                                HiveTableScan(table=[[default, catalog_returns]], table:alias=[catalog_returns])
+                  HiveProject(ib_income_band_sk=[$0])
+                    HiveFilter(condition=[IS NOT NULL($0)])
+                      HiveTableScan(table=[[default, income_band]], table:alias=[ib1])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/cbo_query65.q.out b/ql/src/test/results/clientpositive/perf/tez/cbo_query65.q.out
index ed7b169..e739cb8 100644
--- a/ql/src/test/results/clientpositive/perf/tez/cbo_query65.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/cbo_query65.q.out
@@ -66,34 +66,37 @@ POSTHOOK: Input: default@store_sales
 POSTHOOK: Output: hdfs://### HDFS PATH ###
 CBO PLAN:
 HiveSortLimit(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC], fetch=[100])
-  HiveProject(s_store_name=[$11], i_item_desc=[$1], revenue=[$7], i_current_price=[$2], i_wholesale_cost=[$3], i_brand=[$4])
-    HiveJoin(condition=[=($0, $6)], joinType=[inner], algorithm=[none], cost=[not available])
+  HiveProject(s_store_name=[$6], i_item_desc=[$1], revenue=[$9], i_current_price=[$2], i_wholesale_cost=[$3], i_brand=[$4])
+    HiveJoin(condition=[=($0, $8)], joinType=[inner], algorithm=[none], cost=[not available])
       HiveProject(i_item_sk=[$0], i_item_desc=[$4], i_current_price=[$5], i_wholesale_cost=[$6], i_brand=[$8])
         HiveFilter(condition=[IS NOT NULL($0)])
           HiveTableScan(table=[[default, item]], table:alias=[item])
-      HiveJoin(condition=[=($5, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+      HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+        HiveProject(s_store_sk=[$0], s_store_name=[$5])
+          HiveFilter(condition=[IS NOT NULL($0)])
+            HiveTableScan(table=[[default, store]], table:alias=[store])
         HiveJoin(condition=[AND(=($3, $0), <=($2, $4))], joinType=[inner], algorithm=[none], cost=[not available])
-          HiveProject(ss_store_sk=[$1], ss_item_sk=[$0], $f2=[$2])
-            HiveAggregate(group=[{1, 2}], agg#0=[sum($3)])
-              HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available])
-                HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2], ss_store_sk=[$7], ss_sales_price=[$13])
-                  HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7), IS NOT NULL($2))])
-                    HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                HiveProject(d_date_sk=[$0])
-                  HiveFilter(condition=[AND(BETWEEN(false, $3, 1212, 1223), IS NOT NULL($0))])
-                    HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-          HiveProject($f0=[$0], *=[*(0.1:DECIMAL(2, 1), /($1, $2))])
-            HiveAggregate(group=[{1}], agg#0=[sum($2)], agg#1=[count($2)])
-              HiveProject(ss_item_sk=[$0], ss_store_sk=[$1], $f2=[$2])
+          HiveProject(ss_store_sk=[$0], ss_item_sk=[$1], $f2=[$2])
+            HiveFilter(condition=[IS NOT NULL($2)])
+              HiveProject(ss_store_sk=[$1], ss_item_sk=[$0], $f2=[$2])
                 HiveAggregate(group=[{1, 2}], agg#0=[sum($3)])
                   HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available])
                     HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2], ss_store_sk=[$7], ss_sales_price=[$13])
-                      HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7))])
+                      HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7), IS NOT NULL($2))])
                         HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
                     HiveProject(d_date_sk=[$0])
                       HiveFilter(condition=[AND(BETWEEN(false, $3, 1212, 1223), IS NOT NULL($0))])
                         HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-        HiveProject(s_store_sk=[$0], s_store_name=[$5])
-          HiveFilter(condition=[IS NOT NULL($0)])
-            HiveTableScan(table=[[default, store]], table:alias=[store])
+          HiveProject($f0=[$0], *=[*(0.1:DECIMAL(2, 1), /($1, $2))])
+            HiveFilter(condition=[AND(IS NOT NULL($1), IS NOT NULL($2))])
+              HiveAggregate(group=[{1}], agg#0=[sum($2)], agg#1=[count($2)])
+                HiveProject(ss_item_sk=[$0], ss_store_sk=[$1], $f2=[$2])
+                  HiveAggregate(group=[{1, 2}], agg#0=[sum($3)])
+                    HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available])
+                      HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2], ss_store_sk=[$7], ss_sales_price=[$13])
+                        HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7))])
+                          HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                      HiveProject(d_date_sk=[$0])
+                        HiveFilter(condition=[AND(BETWEEN(false, $3, 1212, 1223), IS NOT NULL($0))])
+                          HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/cbo_query72.q.out b/ql/src/test/results/clientpositive/perf/tez/cbo_query72.q.out
index 30eec8f..129116d 100644
--- a/ql/src/test/results/clientpositive/perf/tez/cbo_query72.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/cbo_query72.q.out
@@ -89,7 +89,7 @@ HiveSortLimit(sort0=[$5], sort1=[$0], sort2=[$1], sort3=[$2], dir0=[DESC-nulls-l
               HiveJoin(condition=[AND(=($14, $1), <($3, $17))], joinType=[inner], algorithm=[none], cost=[not available])
                 HiveJoin(condition=[=($4, $2)], joinType=[inner], algorithm=[none], cost=[not available])
                   HiveProject(inv_date_sk=[$0], inv_item_sk=[$1], inv_warehouse_sk=[$2], inv_quantity_on_hand=[$3])
-                    HiveFilter(condition=[AND(IS NOT NULL($1), IS NOT NULL($2), IS NOT NULL($0))])
+                    HiveFilter(condition=[AND(IS NOT NULL($1), IS NOT NULL($2), IS NOT NULL($0), IS NOT NULL($3))])
                       HiveTableScan(table=[[default, inventory]], table:alias=[inventory])
                   HiveProject(w_warehouse_sk=[$0], w_warehouse_name=[$2])
                     HiveFilter(condition=[IS NOT NULL($0)])
@@ -97,7 +97,7 @@ HiveSortLimit(sort0=[$5], sort1=[$0], sort2=[$1], sort3=[$2], dir0=[DESC-nulls-l
                 HiveProject(d_date_sk=[$0], CAST=[$1], i_item_sk=[$2], i_item_desc=[$3], cs_sold_date_sk=[$4], cs_ship_date_sk=[$5], cs_bill_cdemo_sk=[$6], cs_bill_hdemo_sk=[$7], cs_item_sk=[$8], cs_promo_sk=[$9], cs_order_number=[$10], cs_quantity=[$11], d_date_sk0=[$12], d_week_seq=[$13], +=[$14], cd_demo_sk=[$15], hd_demo_sk=[$16], p_promo_sk=[$17])
                   HiveJoin(condition=[AND(=($5, $0), >($1, $14))], joinType=[inner], algorithm=[none], cost=[not available])
                     HiveProject(d_date_sk=[$0], CAST=[CAST($2):DOUBLE])
-                      HiveFilter(condition=[IS NOT NULL($0)])
+                      HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL(CAST($2):DOUBLE))])
                         HiveTableScan(table=[[default, date_dim]], table:alias=[d3])
                     HiveJoin(condition=[=($0, $6)], joinType=[inner], algorithm=[none], cost=[not available])
                       HiveProject(i_item_sk=[$0], i_item_desc=[$4])
@@ -108,10 +108,10 @@ HiveSortLimit(sort0=[$5], sort1=[$0], sort2=[$1], sort3=[$2], dir0=[DESC-nulls-l
                           HiveJoin(condition=[=($2, $11)], joinType=[inner], algorithm=[none], cost=[not available])
                             HiveJoin(condition=[=($0, $8)], joinType=[inner], algorithm=[none], cost=[not available])
                               HiveProject(cs_sold_date_sk=[$0], cs_ship_date_sk=[$2], cs_bill_cdemo_sk=[$4], cs_bill_hdemo_sk=[$5], cs_item_sk=[$15], cs_promo_sk=[$16], cs_order_number=[$17], cs_quantity=[$18])
-                                HiveFilter(condition=[AND(IS NOT NULL($15), IS NOT NULL($4), IS NOT NULL($5), IS NOT NULL($0), IS NOT NULL($2))])
+                                HiveFilter(condition=[AND(IS NOT NULL($15), IS NOT NULL($4), IS NOT NULL($5), IS NOT NULL($0), IS NOT NULL($2), IS NOT NULL($18))])
                                   HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
                               HiveProject(d_date_sk=[$0], d_week_seq=[$4], +=[+(CAST($2):DOUBLE, 5)])
-                                HiveFilter(condition=[AND(=($6, 2001), IS NOT NULL($0), IS NOT NULL($4))])
+                                HiveFilter(condition=[AND(=($6, 2001), IS NOT NULL($0), IS NOT NULL($4), IS NOT NULL(CAST($2):DOUBLE))])
                                   HiveTableScan(table=[[default, date_dim]], table:alias=[d1])
                             HiveProject(cd_demo_sk=[$0])
                               HiveFilter(condition=[AND(=($2, _UTF-16LE'M'), IS NOT NULL($0))])
diff --git a/ql/src/test/results/clientpositive/perf/tez/cbo_query81.q.out b/ql/src/test/results/clientpositive/perf/tez/cbo_query81.q.out
index b23efd5..3e63c0c 100644
--- a/ql/src/test/results/clientpositive/perf/tez/cbo_query81.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/cbo_query81.q.out
@@ -82,22 +82,9 @@ HiveProject(c_customer_id=[$0], c_salutation=[$1], c_first_name=[$2], c_last_nam
               HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
         HiveProject(cr_returning_customer_sk=[$0], ca_state=[$1], $f2=[$2], _o__c0=[$3], ctr_state=[$4])
           HiveJoin(condition=[AND(=($1, $4), >($2, $3))], joinType=[inner], algorithm=[none], cost=[not available])
-            HiveProject(cr_returning_customer_sk=[$1], ca_state=[$0], $f2=[$2])
-              HiveAggregate(group=[{1, 3}], agg#0=[sum($5)])
-                HiveJoin(condition=[=($4, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                  HiveProject(ca_address_sk=[$0], ca_state=[$8])
-                    HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($8))])
-                      HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
-                  HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available])
-                    HiveProject(cr_returned_date_sk=[$0], cr_returning_customer_sk=[$7], cr_returning_addr_sk=[$10], cr_return_amt_inc_tax=[$20])
-                      HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($10), IS NOT NULL($7))])
-                        HiveTableScan(table=[[default, catalog_returns]], table:alias=[catalog_returns])
-                    HiveProject(d_date_sk=[$0])
-                      HiveFilter(condition=[AND(=($6, 1998), IS NOT NULL($0))])
-                        HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-            HiveProject(_o__c0=[*(/($1, $2), 1.2:DECIMAL(2, 1))], ctr_state=[$0])
-              HiveAggregate(group=[{0}], agg#0=[sum($2)], agg#1=[count($2)])
-                HiveProject(ca_state=[$0], cr_returning_customer_sk=[$1], $f2=[$2])
+            HiveProject(cr_returning_customer_sk=[$0], ca_state=[$1], $f2=[$2])
+              HiveFilter(condition=[IS NOT NULL($2)])
+                HiveProject(cr_returning_customer_sk=[$1], ca_state=[$0], $f2=[$2])
                   HiveAggregate(group=[{1, 3}], agg#0=[sum($5)])
                     HiveJoin(condition=[=($4, $0)], joinType=[inner], algorithm=[none], cost=[not available])
                       HiveProject(ca_address_sk=[$0], ca_state=[$8])
@@ -105,9 +92,25 @@ HiveProject(c_customer_id=[$0], c_salutation=[$1], c_first_name=[$2], c_last_nam
                           HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
                       HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available])
                         HiveProject(cr_returned_date_sk=[$0], cr_returning_customer_sk=[$7], cr_returning_addr_sk=[$10], cr_return_amt_inc_tax=[$20])
-                          HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($10))])
+                          HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($10), IS NOT NULL($7))])
                             HiveTableScan(table=[[default, catalog_returns]], table:alias=[catalog_returns])
                         HiveProject(d_date_sk=[$0])
                           HiveFilter(condition=[AND(=($6, 1998), IS NOT NULL($0))])
                             HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+            HiveProject(_o__c0=[*(/($1, $2), 1.2:DECIMAL(2, 1))], ctr_state=[$0])
+              HiveFilter(condition=[AND(IS NOT NULL($1), IS NOT NULL($2))])
+                HiveAggregate(group=[{0}], agg#0=[sum($2)], agg#1=[count($2)])
+                  HiveProject(ca_state=[$0], cr_returning_customer_sk=[$1], $f2=[$2])
+                    HiveAggregate(group=[{1, 3}], agg#0=[sum($5)])
+                      HiveJoin(condition=[=($4, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+                        HiveProject(ca_address_sk=[$0], ca_state=[$8])
+                          HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($8))])
+                            HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
+                        HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available])
+                          HiveProject(cr_returned_date_sk=[$0], cr_returning_customer_sk=[$7], cr_returning_addr_sk=[$10], cr_return_amt_inc_tax=[$20])
+                            HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($10))])
+                              HiveTableScan(table=[[default, catalog_returns]], table:alias=[catalog_returns])
+                          HiveProject(d_date_sk=[$0])
+                            HiveFilter(condition=[AND(=($6, 1998), IS NOT NULL($0))])
+                              HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/cbo_query92.q.out b/ql/src/test/results/clientpositive/perf/tez/cbo_query92.q.out
index ece079f..1a45e85 100644
--- a/ql/src/test/results/clientpositive/perf/tez/cbo_query92.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/cbo_query92.q.out
@@ -66,26 +66,26 @@ POSTHOOK: Input: default@web_sales
 POSTHOOK: Output: hdfs://### HDFS PATH ###
 CBO PLAN:
 HiveAggregate(group=[{}], agg#0=[sum($2)])
-  HiveJoin(condition=[AND(>($2, $5), =($6, $1))], joinType=[inner], algorithm=[none], cost=[not available])
-    HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-      HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$3], ws_ext_discount_amt=[$22])
-        HiveFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($0))])
-          HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
-      HiveProject(d_date_sk=[$0])
-        HiveFilter(condition=[AND(BETWEEN(false, CAST($2):TIMESTAMP(9), 1998-03-18 00:00:00:TIMESTAMP(9), 1998-06-16 00:00:00:TIMESTAMP(9)), IS NOT NULL($0))])
-          HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-    HiveProject(ws_item_sk=[$0], CAST=[$1], i_item_sk=[$2])
-      HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-        HiveProject(ws_item_sk=[$0], CAST=[CAST(*(1.3:DECIMAL(2, 1), /($1, $2))):DECIMAL(14, 7)])
-          HiveAggregate(group=[{1}], agg#0=[sum($2)], agg#1=[count($2)])
-            HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-              HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$3], ws_ext_discount_amt=[$22])
-                HiveFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($0))])
-                  HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
-              HiveProject(d_date_sk=[$0])
-                HiveFilter(condition=[AND(BETWEEN(false, CAST($2):TIMESTAMP(9), 1998-03-18 00:00:00:TIMESTAMP(9), 1998-06-16 00:00:00:TIMESTAMP(9)), IS NOT NULL($0))])
-                  HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-        HiveProject(i_item_sk=[$0])
-          HiveFilter(condition=[AND(=($13, 269), IS NOT NULL($0))])
-            HiveTableScan(table=[[default, item]], table:alias=[item])
+  HiveJoin(condition=[AND(=($5, $4), >($2, $6))], joinType=[inner], algorithm=[none], cost=[not available])
+    HiveJoin(condition=[=($4, $1)], joinType=[inner], algorithm=[none], cost=[not available])
+      HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+        HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$3], ws_ext_discount_amt=[$22])
+          HiveFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($0), IS NOT NULL($22))])
+            HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
+        HiveProject(d_date_sk=[$0])
+          HiveFilter(condition=[AND(BETWEEN(false, CAST($2):TIMESTAMP(9), 1998-03-18 00:00:00:TIMESTAMP(9), 1998-06-16 00:00:00:TIMESTAMP(9)), IS NOT NULL($0))])
+            HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+      HiveProject(i_item_sk=[$0])
+        HiveFilter(condition=[AND(=($13, 269), IS NOT NULL($0))])
+          HiveTableScan(table=[[default, item]], table:alias=[item])
+    HiveProject(ws_item_sk=[$0], CAST=[CAST(*(1.3:DECIMAL(2, 1), /($1, $2))):DECIMAL(14, 7)])
+      HiveFilter(condition=[IS NOT NULL(CAST(*(1.3:DECIMAL(2, 1), /($1, $2))):DECIMAL(14, 7))])
+        HiveAggregate(group=[{1}], agg#0=[sum($2)], agg#1=[count($2)])
+          HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+            HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$3], ws_ext_discount_amt=[$22])
+              HiveFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($0))])
+                HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
+            HiveProject(d_date_sk=[$0])
+              HiveFilter(condition=[AND(BETWEEN(false, CAST($2):TIMESTAMP(9), 1998-03-18 00:00:00:TIMESTAMP(9), 1998-06-16 00:00:00:TIMESTAMP(9)), IS NOT NULL($0))])
+                HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_ext_query1.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_ext_query1.q.out
index 41783ce..7b754f7 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_ext_query1.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_ext_query1.q.out
@@ -59,34 +59,36 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 CBO PLAN:
 HiveSortLimit(sort0=[$0], dir0=[ASC], fetch=[100]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
   HiveProject(c_customer_id=[$1]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-    HiveJoin(condition=[=($2, $0)], joinType=[inner], algorithm=[none], cost=[not available]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+    HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
       HiveProject(c_customer_sk=[$0], c_customer_id=[$1]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
         HiveTableScan(table=[[default, customer]], table:alias=[customer]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-      HiveJoin(condition=[AND(=($1, $5), >($2, $4))], joinType=[inner], algorithm=[none], cost=[not available]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-        HiveJoin(condition=[=($3, $1)], joinType=[inner], algorithm=[none], cost=[not available]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-          HiveProject(sr_customer_sk=[$0], sr_store_sk=[$1], $f2=[$2]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-            HiveAggregate(group=[{1, 2}], agg#0=[sum($3)]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-              HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-                HiveProject(sr_returned_date_sk=[$0], sr_customer_sk=[$3], sr_store_sk=[$7], sr_fee=[$14]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-                  HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7), IS NOT NULL($3))]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-                    HiveTableScan(table=[[default, store_returns]], table:alias=[store_returns]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-                HiveProject(d_date_sk=[$0]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-                  HiveFilter(condition=[=($6, 2000)]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-                    HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+      HiveJoin(condition=[AND(=($2, $5), >($3, $4))], joinType=[inner], algorithm=[none], cost=[not available]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+        HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
           HiveProject(s_store_sk=[$0]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
             HiveFilter(condition=[=($24, _UTF-16LE'NM')]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
               HiveTableScan(table=[[default, store]], table:alias=[store]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-        HiveProject(_o__c0=[*(/($1, $2), 1.2:DECIMAL(2, 1))], ctr_store_sk=[$0]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-          HiveAggregate(group=[{1}], agg#0=[sum($2)], agg#1=[count($2)]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-            HiveProject(sr_customer_sk=[$0], sr_store_sk=[$1], $f2=[$2]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+          HiveProject(sr_customer_sk=[$0], sr_store_sk=[$1], $f2=[$2]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+            HiveFilter(condition=[IS NOT NULL($2)]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
               HiveAggregate(group=[{1, 2}], agg#0=[sum($3)]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
                 HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
                   HiveProject(sr_returned_date_sk=[$0], sr_customer_sk=[$3], sr_store_sk=[$7], sr_fee=[$14]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-                    HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7))]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+                    HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7), IS NOT NULL($3))]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
                       HiveTableScan(table=[[default, store_returns]], table:alias=[store_returns]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
                   HiveProject(d_date_sk=[$0]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
                     HiveFilter(condition=[=($6, 2000)]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
                       HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+        HiveProject(_o__c0=[*(/($1, $2), 1.2:DECIMAL(2, 1))], ctr_store_sk=[$0]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+          HiveFilter(condition=[AND(IS NOT NULL($1), IS NOT NULL($2))]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+            HiveAggregate(group=[{1}], agg#0=[sum($2)], agg#1=[count($2)]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+              HiveProject(sr_customer_sk=[$0], sr_store_sk=[$1], $f2=[$2]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+                HiveAggregate(group=[{1, 2}], agg#0=[sum($3)]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+                  HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+                    HiveProject(sr_returned_date_sk=[$0], sr_customer_sk=[$3], sr_store_sk=[$7], sr_fee=[$14]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+                      HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7))]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+                        HiveTableScan(table=[[default, store_returns]], table:alias=[store_returns]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+                    HiveProject(d_date_sk=[$0]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+                      HiveFilter(condition=[=($6, 2000)]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+                        HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
 
 PREHOOK: query: explain cbo joincost
 with customer_total_return as
@@ -149,32 +151,34 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 CBO PLAN:
 HiveSortLimit(sort0=[$0], dir0=[ASC], fetch=[100]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
   HiveProject(c_customer_id=[$1]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-    HiveJoin(condition=[=($2, $0)], joinType=[inner], algorithm=[none], cost=[{8.041115324461429E7 rows, 0.0 cpu, 0.0 io}]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+    HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[{8.036998822876796E7 rows, 0.0 cpu, 0.0 io}]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
       HiveProject(c_customer_sk=[$0], c_customer_id=[$1]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
         HiveTableScan(table=[[default, customer]], table:alias=[customer]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-      HiveJoin(condition=[AND(=($1, $5), >($2, $4))], joinType=[inner], algorithm=[none], cost=[{415687.382770037 rows, 0.0 cpu, 0.0 io}]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-        HiveJoin(condition=[=($3, $1)], joinType=[inner], algorithm=[none], cost=[{460301.9976112889 rows, 0.0 cpu, 0.0 io}]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-          HiveProject(sr_customer_sk=[$0], sr_store_sk=[$1], $f2=[$2]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-            HiveAggregate(group=[{1, 2}], agg#0=[sum($3)]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-              HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[{5.175767820386722E7 rows, 0.0 cpu, 0.0 io}]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-                HiveProject(sr_returned_date_sk=[$0], sr_customer_sk=[$3], sr_store_sk=[$7], sr_fee=[$14]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-                  HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7), IS NOT NULL($3))]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-                    HiveTableScan(table=[[default, store_returns]], table:alias=[store_returns]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-                HiveProject(d_date_sk=[$0]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-                  HiveFilter(condition=[=($6, 2000)]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-                    HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+      HiveJoin(condition=[AND(=($2, $5), >($3, $4))], joinType=[inner], algorithm=[none], cost=[{374952.69636986067 rows, 0.0 cpu, 0.0 io}]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+        HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[{455187.9173657213 rows, 0.0 cpu, 0.0 io}]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
           HiveProject(s_store_sk=[$0]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
             HiveFilter(condition=[=($24, _UTF-16LE'NM')]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
               HiveTableScan(table=[[default, store]], table:alias=[store]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-        HiveProject(_o__c0=[*(/($1, $2), 1.2:DECIMAL(2, 1))], ctr_store_sk=[$0]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-          HiveAggregate(group=[{1}], agg#0=[sum($2)], agg#1=[count($2)]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-            HiveProject(sr_customer_sk=[$0], sr_store_sk=[$1], $f2=[$2]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+          HiveProject(sr_customer_sk=[$0], sr_store_sk=[$1], $f2=[$2]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+            HiveFilter(condition=[IS NOT NULL($2)]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
               HiveAggregate(group=[{1, 2}], agg#0=[sum($3)]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-                HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[{5.3635511784936875E7 rows, 0.0 cpu, 0.0 io}]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+                HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[{5.175767820386722E7 rows, 0.0 cpu, 0.0 io}]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
                   HiveProject(sr_returned_date_sk=[$0], sr_customer_sk=[$3], sr_store_sk=[$7], sr_fee=[$14]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
-                    HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7))]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+                    HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7), IS NOT NULL($3))]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
                       HiveTableScan(table=[[default, store_returns]], table:alias=[store_returns]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
                   HiveProject(d_date_sk=[$0]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
                     HiveFilter(condition=[=($6, 2000)]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
                       HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+        HiveProject(_o__c0=[*(/($1, $2), 1.2:DECIMAL(2, 1))], ctr_store_sk=[$0]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+          HiveFilter(condition=[AND(IS NOT NULL($1), IS NOT NULL($2))]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+            HiveAggregate(group=[{1}], agg#0=[sum($2)], agg#1=[count($2)]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+              HiveProject(sr_customer_sk=[$0], sr_store_sk=[$1], $f2=[$2]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+                HiveAggregate(group=[{1, 2}], agg#0=[sum($3)]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+                  HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[{5.3635511784936875E7 rows, 0.0 cpu, 0.0 io}]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+                    HiveProject(sr_returned_date_sk=[$0], sr_customer_sk=[$3], sr_store_sk=[$7], sr_fee=[$14]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+                      HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7))]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+                        HiveTableScan(table=[[default, store_returns]], table:alias=[store_returns]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+                    HiveProject(d_date_sk=[$0]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+                      HiveFilter(condition=[=($6, 2000)]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
+                        HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim]): rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query1.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query1.q.out
index 67bf09e..d506914 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query1.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query1.q.out
@@ -59,32 +59,34 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 CBO PLAN:
 HiveSortLimit(sort0=[$0], dir0=[ASC], fetch=[100])
   HiveProject(c_customer_id=[$1])
-    HiveJoin(condition=[=($2, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+    HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
       HiveProject(c_customer_sk=[$0], c_customer_id=[$1])
         HiveTableScan(table=[[default, customer]], table:alias=[customer])
-      HiveJoin(condition=[AND(=($1, $5), >($2, $4))], joinType=[inner], algorithm=[none], cost=[not available])
-        HiveJoin(condition=[=($3, $1)], joinType=[inner], algorithm=[none], cost=[not available])
-          HiveProject(sr_customer_sk=[$0], sr_store_sk=[$1], $f2=[$2])
-            HiveAggregate(group=[{1, 2}], agg#0=[sum($3)])
-              HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available])
-                HiveProject(sr_returned_date_sk=[$0], sr_customer_sk=[$3], sr_store_sk=[$7], sr_fee=[$14])
-                  HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7), IS NOT NULL($3))])
-                    HiveTableScan(table=[[default, store_returns]], table:alias=[store_returns])
-                HiveProject(d_date_sk=[$0])
-                  HiveFilter(condition=[=($6, 2000)])
-                    HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+      HiveJoin(condition=[AND(=($2, $5), >($3, $4))], joinType=[inner], algorithm=[none], cost=[not available])
+        HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
           HiveProject(s_store_sk=[$0])
             HiveFilter(condition=[=($24, _UTF-16LE'NM')])
               HiveTableScan(table=[[default, store]], table:alias=[store])
-        HiveProject(_o__c0=[*(/($1, $2), 1.2:DECIMAL(2, 1))], ctr_store_sk=[$0])
-          HiveAggregate(group=[{1}], agg#0=[sum($2)], agg#1=[count($2)])
-            HiveProject(sr_customer_sk=[$0], sr_store_sk=[$1], $f2=[$2])
+          HiveProject(sr_customer_sk=[$0], sr_store_sk=[$1], $f2=[$2])
+            HiveFilter(condition=[IS NOT NULL($2)])
               HiveAggregate(group=[{1, 2}], agg#0=[sum($3)])
                 HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available])
                   HiveProject(sr_returned_date_sk=[$0], sr_customer_sk=[$3], sr_store_sk=[$7], sr_fee=[$14])
-                    HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7))])
+                    HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7), IS NOT NULL($3))])
                       HiveTableScan(table=[[default, store_returns]], table:alias=[store_returns])
                   HiveProject(d_date_sk=[$0])
                     HiveFilter(condition=[=($6, 2000)])
                       HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+        HiveProject(_o__c0=[*(/($1, $2), 1.2:DECIMAL(2, 1))], ctr_store_sk=[$0])
+          HiveFilter(condition=[AND(IS NOT NULL($1), IS NOT NULL($2))])
+            HiveAggregate(group=[{1}], agg#0=[sum($2)], agg#1=[count($2)])
+              HiveProject(sr_customer_sk=[$0], sr_store_sk=[$1], $f2=[$2])
+                HiveAggregate(group=[{1, 2}], agg#0=[sum($3)])
+                  HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available])
+                    HiveProject(sr_returned_date_sk=[$0], sr_customer_sk=[$3], sr_store_sk=[$7], sr_fee=[$14])
+                      HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7))])
+                        HiveTableScan(table=[[default, store_returns]], table:alias=[store_returns])
+                    HiveProject(d_date_sk=[$0])
+                      HiveFilter(condition=[=($6, 2000)])
+                        HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query14.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query14.q.out
index 2af220a..7eacbc7 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query14.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query14.q.out
@@ -1,6 +1,6 @@
-Warning: Shuffle Join MERGEJOIN[1182][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 6' is a cross product
-Warning: Shuffle Join MERGEJOIN[1189][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 16' is a cross product
-Warning: Shuffle Join MERGEJOIN[1196][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 22' is a cross product
+Warning: Shuffle Join MERGEJOIN[1191][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 6' is a cross product
+Warning: Shuffle Join MERGEJOIN[1198][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 16' is a cross product
+Warning: Shuffle Join MERGEJOIN[1205][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 22' is a cross product
 PREHOOK: query: explain cbo
 with  cross_items as
  (select i_item_sk ss_item_sk
@@ -228,280 +228,289 @@ HiveSortLimit(sort0=[$0], sort1=[$1], sort2=[$2], sort3=[$3], dir0=[ASC], dir1=[
           HiveProject(channel=[_UTF-16LE'store':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"], i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], sales=[$3], number_sales=[$4])
             HiveJoin(condition=[>($3, $5)], joinType=[inner], algorithm=[none], cost=[not available])
               HiveProject($f0=[$0], $f1=[$1], $f2=[$2], $f3=[$3], $f4=[$4])
-                HiveAggregate(group=[{0, 1, 2}], agg#0=[sum($3)], agg#1=[count()])
-                  HiveProject($f0=[$1], $f1=[$2], $f2=[$3], $f3=[*(CAST($7):DECIMAL(10, 0), $8)])
-                    HiveJoin(condition=[=($6, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                      HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
-                        HiveTableScan(table=[[default, item]], table:alias=[item])
-                      HiveJoin(condition=[=($2, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                        HiveProject(ss_item_sk=[$0])
-                          HiveJoin(condition=[AND(=($1, $4), =($2, $5), =($3, $6))], joinType=[inner], algorithm=[none], cost=[not available])
-                            HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
-                              HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11))])
-                                HiveTableScan(table=[[default, item]], table:alias=[item])
-                            HiveProject($f0=[$0], $f1=[$1], $f2=[$2])
-                              HiveFilter(condition=[=($3, 3)])
-                                HiveAggregate(group=[{0, 1, 2}], agg#0=[count($3)])
-                                  HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3])
-                                    HiveUnion(all=[true])
-                                      HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3])
-                                        HiveAggregate(group=[{4, 5, 6}], agg#0=[count()])
-                                          HiveJoin(condition=[=($1, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                                            HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-                                              HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2])
-                                                HiveFilter(condition=[IS NOT NULL($0)])
-                                                  HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                                              HiveProject(d_date_sk=[$0])
-                                                HiveFilter(condition=[BETWEEN(false, $6, 1999, 2001)])
-                                                  HiveTableScan(table=[[default, date_dim]], table:alias=[d1])
-                                            HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
-                                              HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11))])
-                                                HiveTableScan(table=[[default, item]], table:alias=[iss])
-                                      HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3])
-                                        HiveAggregate(group=[{4, 5, 6}], agg#0=[count()])
-                                          HiveJoin(condition=[=($1, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                                            HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-                                              HiveProject(cs_sold_date_sk=[$0], cs_item_sk=[$15])
-                                                HiveFilter(condition=[IS NOT NULL($0)])
-                                                  HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
-                                              HiveProject(d_date_sk=[$0])
-                                                HiveFilter(condition=[BETWEEN(false, $6, 1999, 2001)])
-                                                  HiveTableScan(table=[[default, date_dim]], table:alias=[d2])
-                                            HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
-                                              HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11))])
-                                                HiveTableScan(table=[[default, item]], table:alias=[ics])
-                                      HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3])
-                                        HiveAggregate(group=[{4, 5, 6}], agg#0=[count()])
-                                          HiveJoin(condition=[=($1, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                                            HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-                                              HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$3])
-                                                HiveFilter(condition=[IS NOT NULL($0)])
-                                                  HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
-                                              HiveProject(d_date_sk=[$0])
-                                                HiveFilter(condition=[BETWEEN(false, $6, 1999, 2001)])
-                                                  HiveTableScan(table=[[default, date_dim]], table:alias=[d3])
-                                            HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
-                                              HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11))])
-                                                HiveTableScan(table=[[default, item]], table:alias=[iws])
-                        HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available])
-                          HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2], ss_quantity=[$10], ss_list_price=[$12])
-                            HiveFilter(condition=[IS NOT NULL($0)])
-                              HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                          HiveProject(d_date_sk=[$0])
-                            HiveFilter(condition=[AND(=($6, 2000), =($8, 11))])
-                              HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+                HiveFilter(condition=[IS NOT NULL($3)])
+                  HiveAggregate(group=[{0, 1, 2}], agg#0=[sum($3)], agg#1=[count()])
+                    HiveProject($f0=[$1], $f1=[$2], $f2=[$3], $f3=[*(CAST($7):DECIMAL(10, 0), $8)])
+                      HiveJoin(condition=[=($6, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+                        HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
+                          HiveTableScan(table=[[default, item]], table:alias=[item])
+                        HiveJoin(condition=[=($2, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+                          HiveProject(ss_item_sk=[$0])
+                            HiveJoin(condition=[AND(=($1, $4), =($2, $5), =($3, $6))], joinType=[inner], algorithm=[none], cost=[not available])
+                              HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
+                                HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11))])
+                                  HiveTableScan(table=[[default, item]], table:alias=[item])
+                              HiveProject($f0=[$0], $f1=[$1], $f2=[$2])
+                                HiveFilter(condition=[=($3, 3)])
+                                  HiveAggregate(group=[{0, 1, 2}], agg#0=[count($3)])
+                                    HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3])
+                                      HiveUnion(all=[true])
+                                        HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3])
+                                          HiveAggregate(group=[{4, 5, 6}], agg#0=[count()])
+                                            HiveJoin(condition=[=($1, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                                              HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+                                                HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2])
+                                                  HiveFilter(condition=[IS NOT NULL($0)])
+                                                    HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                                                HiveProject(d_date_sk=[$0])
+                                                  HiveFilter(condition=[BETWEEN(false, $6, 1999, 2001)])
+                                                    HiveTableScan(table=[[default, date_dim]], table:alias=[d1])
+                                              HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
+                                                HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11))])
+                                                  HiveTableScan(table=[[default, item]], table:alias=[iss])
+                                        HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3])
+                                          HiveAggregate(group=[{4, 5, 6}], agg#0=[count()])
+                                            HiveJoin(condition=[=($1, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                                              HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+                                                HiveProject(cs_sold_date_sk=[$0], cs_item_sk=[$15])
+                                                  HiveFilter(condition=[IS NOT NULL($0)])
+                                                    HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
+                                                HiveProject(d_date_sk=[$0])
+                                                  HiveFilter(condition=[BETWEEN(false, $6, 1999, 2001)])
+                                                    HiveTableScan(table=[[default, date_dim]], table:alias=[d2])
+                                              HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
+                                                HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11))])
+                                                  HiveTableScan(table=[[default, item]], table:alias=[ics])
+                                        HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3])
+                                          HiveAggregate(group=[{4, 5, 6}], agg#0=[count()])
+                                            HiveJoin(condition=[=($1, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                                              HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+                                                HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$3])
+                                                  HiveFilter(condition=[IS NOT NULL($0)])
+                                                    HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
+                                                HiveProject(d_date_sk=[$0])
+                                                  HiveFilter(condition=[BETWEEN(false, $6, 1999, 2001)])
+                                                    HiveTableScan(table=[[default, date_dim]], table:alias=[d3])
+                                              HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
+                                                HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11))])
+                                                  HiveTableScan(table=[[default, item]], table:alias=[iws])
+                          HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available])
+                            HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2], ss_quantity=[$10], ss_list_price=[$12])
+                              HiveFilter(condition=[IS NOT NULL($0)])
+                                HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                            HiveProject(d_date_sk=[$0])
+                              HiveFilter(condition=[AND(=($6, 2000), =($8, 11))])
+                                HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
               HiveProject($f0=[/($0, $1)])
-                HiveAggregate(group=[{}], agg#0=[sum($0)], agg#1=[count($0)])
-                  HiveProject($f0=[*(CAST($0):DECIMAL(10, 0), $1)])
-                    HiveUnion(all=[true])
-                      HiveProject(quantity=[$1], list_price=[$2])
-                        HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                          HiveProject(ss_sold_date_sk=[$0], ss_quantity=[$10], ss_list_price=[$12])
-                            HiveFilter(condition=[IS NOT NULL($0)])
-                              HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                          HiveProject(d_date_sk=[$0])
-                            HiveFilter(condition=[BETWEEN(false, $6, 1999, 2001)])
-                              HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-                      HiveProject(quantity=[$1], list_price=[$2])
-                        HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                          HiveProject(cs_sold_date_sk=[$0], cs_quantity=[$18], cs_list_price=[$20])
-                            HiveFilter(condition=[IS NOT NULL($0)])
-                              HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
-                          HiveProject(d_date_sk=[$0])
-                            HiveFilter(condition=[BETWEEN(false, $6, 1998, 2000)])
-                              HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-                      HiveProject(quantity=[$1], list_price=[$2])
-                        HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                          HiveProject(ws_sold_date_sk=[$0], ws_quantity=[$18], ws_list_price=[$20])
-                            HiveFilter(condition=[IS NOT NULL($0)])
-                              HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
-                          HiveProject(d_date_sk=[$0])
-                            HiveFilter(condition=[BETWEEN(false, $6, 1998, 2000)])
-                              HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+                HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($1))])
+                  HiveProject($f0=[$0], $f1=[$1])
+                    HiveAggregate(group=[{}], agg#0=[sum($0)], agg#1=[count($0)])
+                      HiveProject($f0=[*(CAST($0):DECIMAL(10, 0), $1)])
+                        HiveUnion(all=[true])
+                          HiveProject(quantity=[$1], list_price=[$2])
+                            HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                              HiveProject(ss_sold_date_sk=[$0], ss_quantity=[$10], ss_list_price=[$12])
+                                HiveFilter(condition=[IS NOT NULL($0)])
+                                  HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                              HiveProject(d_date_sk=[$0])
+                                HiveFilter(condition=[BETWEEN(false, $6, 1999, 2001)])
+                                  HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+                          HiveProject(quantity=[$1], list_price=[$2])
+                            HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                              HiveProject(cs_sold_date_sk=[$0], cs_quantity=[$18], cs_list_price=[$20])
+                                HiveFilter(condition=[IS NOT NULL($0)])
+                                  HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
+                              HiveProject(d_date_sk=[$0])
+                                HiveFilter(condition=[BETWEEN(false, $6, 1998, 2000)])
+                                  HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+                          HiveProject(quantity=[$1], list_price=[$2])
+                            HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                              HiveProject(ws_sold_date_sk=[$0], ws_quantity=[$18], ws_list_price=[$20])
+                                HiveFilter(condition=[IS NOT NULL($0)])
+                                  HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
+                              HiveProject(d_date_sk=[$0])
+                                HiveFilter(condition=[BETWEEN(false, $6, 1998, 2000)])
+                                  HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
           HiveProject(channel=[_UTF-16LE'catalog':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"], i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], sales=[$3], number_sales=[$4])
             HiveJoin(condition=[>($3, $5)], joinType=[inner], algorithm=[none], cost=[not available])
               HiveProject($f0=[$0], $f1=[$1], $f2=[$2], $f3=[$3], $f4=[$4])
-                HiveAggregate(group=[{0, 1, 2}], agg#0=[sum($3)], agg#1=[count()])
-                  HiveProject($f0=[$1], $f1=[$2], $f2=[$3], $f3=[*(CAST($7):DECIMAL(10, 0), $8)])
-                    HiveJoin(condition=[=($6, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                      HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
-                        HiveTableScan(table=[[default, item]], table:alias=[item])
-                      HiveJoin(condition=[=($2, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                        HiveProject(ss_item_sk=[$0])
-                          HiveJoin(condition=[AND(=($1, $4), =($2, $5), =($3, $6))], joinType=[inner], algorithm=[none], cost=[not available])
-                            HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
-                              HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11))])
-                                HiveTableScan(table=[[default, item]], table:alias=[item])
-                            HiveProject($f0=[$0], $f1=[$1], $f2=[$2])
-                              HiveFilter(condition=[=($3, 3)])
-                                HiveAggregate(group=[{0, 1, 2}], agg#0=[count($3)])
-                                  HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3])
-                                    HiveUnion(all=[true])
-                                      HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3])
-                                        HiveAggregate(group=[{4, 5, 6}], agg#0=[count()])
-                                          HiveJoin(condition=[=($1, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                                            HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-                                              HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2])
-                                                HiveFilter(condition=[IS NOT NULL($0)])
-                                                  HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                                              HiveProject(d_date_sk=[$0])
-                                                HiveFilter(condition=[BETWEEN(false, $6, 1999, 2001)])
-                                                  HiveTableScan(table=[[default, date_dim]], table:alias=[d1])
-                                            HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
-                                              HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11))])
-                                                HiveTableScan(table=[[default, item]], table:alias=[iss])
-                                      HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3])
-                                        HiveAggregate(group=[{4, 5, 6}], agg#0=[count()])
-                                          HiveJoin(condition=[=($1, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                                            HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-                                              HiveProject(cs_sold_date_sk=[$0], cs_item_sk=[$15])
-                                                HiveFilter(condition=[IS NOT NULL($0)])
-                                                  HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
-                                              HiveProject(d_date_sk=[$0])
-                                                HiveFilter(condition=[BETWEEN(false, $6, 1999, 2001)])
-                                                  HiveTableScan(table=[[default, date_dim]], table:alias=[d2])
-                                            HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
-                                              HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11))])
-                                                HiveTableScan(table=[[default, item]], table:alias=[ics])
-                                      HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3])
-                                        HiveAggregate(group=[{4, 5, 6}], agg#0=[count()])
-                                          HiveJoin(condition=[=($1, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                                            HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-                                              HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$3])
-                                                HiveFilter(condition=[IS NOT NULL($0)])
-                                                  HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
-                                              HiveProject(d_date_sk=[$0])
-                                                HiveFilter(condition=[BETWEEN(false, $6, 1999, 2001)])
-                                                  HiveTableScan(table=[[default, date_dim]], table:alias=[d3])
-                                            HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
-                                              HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11))])
-                                                HiveTableScan(table=[[default, item]], table:alias=[iws])
-                        HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available])
-                          HiveProject(cs_sold_date_sk=[$0], cs_item_sk=[$15], cs_quantity=[$18], cs_list_price=[$20])
-                            HiveFilter(condition=[IS NOT NULL($0)])
-                              HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
-                          HiveProject(d_date_sk=[$0])
-                            HiveFilter(condition=[AND(=($6, 2000), =($8, 11))])
-                              HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+                HiveFilter(condition=[IS NOT NULL($3)])
+                  HiveAggregate(group=[{0, 1, 2}], agg#0=[sum($3)], agg#1=[count()])
+                    HiveProject($f0=[$1], $f1=[$2], $f2=[$3], $f3=[*(CAST($7):DECIMAL(10, 0), $8)])
+                      HiveJoin(condition=[=($6, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+                        HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
+                          HiveTableScan(table=[[default, item]], table:alias=[item])
+                        HiveJoin(condition=[=($2, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+                          HiveProject(ss_item_sk=[$0])
+                            HiveJoin(condition=[AND(=($1, $4), =($2, $5), =($3, $6))], joinType=[inner], algorithm=[none], cost=[not available])
+                              HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
+                                HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11))])
+                                  HiveTableScan(table=[[default, item]], table:alias=[item])
+                              HiveProject($f0=[$0], $f1=[$1], $f2=[$2])
+                                HiveFilter(condition=[=($3, 3)])
+                                  HiveAggregate(group=[{0, 1, 2}], agg#0=[count($3)])
+                                    HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3])
+                                      HiveUnion(all=[true])
+                                        HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3])
+                                          HiveAggregate(group=[{4, 5, 6}], agg#0=[count()])
+                                            HiveJoin(condition=[=($1, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                                              HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+                                                HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2])
+                                                  HiveFilter(condition=[IS NOT NULL($0)])
+                                                    HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                                                HiveProject(d_date_sk=[$0])
+                                                  HiveFilter(condition=[BETWEEN(false, $6, 1999, 2001)])
+                                                    HiveTableScan(table=[[default, date_dim]], table:alias=[d1])
+                                              HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
+                                                HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11))])
+                                                  HiveTableScan(table=[[default, item]], table:alias=[iss])
+                                        HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3])
+                                          HiveAggregate(group=[{4, 5, 6}], agg#0=[count()])
+                                            HiveJoin(condition=[=($1, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                                              HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+                                                HiveProject(cs_sold_date_sk=[$0], cs_item_sk=[$15])
+                                                  HiveFilter(condition=[IS NOT NULL($0)])
+                                                    HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
+                                                HiveProject(d_date_sk=[$0])
+                                                  HiveFilter(condition=[BETWEEN(false, $6, 1999, 2001)])
+                                                    HiveTableScan(table=[[default, date_dim]], table:alias=[d2])
+                                              HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
+                                                HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11))])
+                                                  HiveTableScan(table=[[default, item]], table:alias=[ics])
+                                        HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3])
+                                          HiveAggregate(group=[{4, 5, 6}], agg#0=[count()])
+                                            HiveJoin(condition=[=($1, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                                              HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+                                                HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$3])
+                                                  HiveFilter(condition=[IS NOT NULL($0)])
+                                                    HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
+                                                HiveProject(d_date_sk=[$0])
+                                                  HiveFilter(condition=[BETWEEN(false, $6, 1999, 2001)])
+                                                    HiveTableScan(table=[[default, date_dim]], table:alias=[d3])
+                                              HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
+                                                HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11))])
+                                                  HiveTableScan(table=[[default, item]], table:alias=[iws])
+                          HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available])
+                            HiveProject(cs_sold_date_sk=[$0], cs_item_sk=[$15], cs_quantity=[$18], cs_list_price=[$20])
+                              HiveFilter(condition=[IS NOT NULL($0)])
+                                HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
+                            HiveProject(d_date_sk=[$0])
+                              HiveFilter(condition=[AND(=($6, 2000), =($8, 11))])
+                                HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
               HiveProject($f0=[/($0, $1)])
-                HiveAggregate(group=[{}], agg#0=[sum($0)], agg#1=[count($0)])
-                  HiveProject($f0=[*(CAST($0):DECIMAL(10, 0), $1)])
-                    HiveUnion(all=[true])
-                      HiveProject(quantity=[$1], list_price=[$2])
-                        HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                          HiveProject(ss_sold_date_sk=[$0], ss_quantity=[$10], ss_list_price=[$12])
-                            HiveFilter(condition=[IS NOT NULL($0)])
-                              HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                          HiveProject(d_date_sk=[$0])
-                            HiveFilter(condition=[BETWEEN(false, $6, 1999, 2001)])
-                              HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-                      HiveProject(quantity=[$1], list_price=[$2])
-                        HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                          HiveProject(cs_sold_date_sk=[$0], cs_quantity=[$18], cs_list_price=[$20])
-                            HiveFilter(condition=[IS NOT NULL($0)])
-                              HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
-                          HiveProject(d_date_sk=[$0])
-                            HiveFilter(condition=[BETWEEN(false, $6, 1998, 2000)])
-                              HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-                      HiveProject(quantity=[$1], list_price=[$2])
-                        HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                          HiveProject(ws_sold_date_sk=[$0], ws_quantity=[$18], ws_list_price=[$20])
-                            HiveFilter(condition=[IS NOT NULL($0)])
-                              HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
-                          HiveProject(d_date_sk=[$0])
-                            HiveFilter(condition=[BETWEEN(false, $6, 1998, 2000)])
-                              HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+                HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($1))])
+                  HiveProject($f0=[$0], $f1=[$1])
+                    HiveAggregate(group=[{}], agg#0=[sum($0)], agg#1=[count($0)])
+                      HiveProject($f0=[*(CAST($0):DECIMAL(10, 0), $1)])
+                        HiveUnion(all=[true])
+                          HiveProject(quantity=[$1], list_price=[$2])
+                            HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                              HiveProject(ss_sold_date_sk=[$0], ss_quantity=[$10], ss_list_price=[$12])
+                                HiveFilter(condition=[IS NOT NULL($0)])
+                                  HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                              HiveProject(d_date_sk=[$0])
+                                HiveFilter(condition=[BETWEEN(false, $6, 1999, 2001)])
+                                  HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+                          HiveProject(quantity=[$1], list_price=[$2])
+                            HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                              HiveProject(cs_sold_date_sk=[$0], cs_quantity=[$18], cs_list_price=[$20])
+                                HiveFilter(condition=[IS NOT NULL($0)])
+                                  HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
+                              HiveProject(d_date_sk=[$0])
+                                HiveFilter(condition=[BETWEEN(false, $6, 1998, 2000)])
+                                  HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+                          HiveProject(quantity=[$1], list_price=[$2])
+                            HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                              HiveProject(ws_sold_date_sk=[$0], ws_quantity=[$18], ws_list_price=[$20])
+                                HiveFilter(condition=[IS NOT NULL($0)])
+                                  HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
+                              HiveProject(d_date_sk=[$0])
+                                HiveFilter(condition=[BETWEEN(false, $6, 1998, 2000)])
+                                  HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
           HiveProject(channel=[_UTF-16LE'web':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"], i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], sales=[$3], number_sales=[$4])
             HiveJoin(condition=[>($3, $5)], joinType=[inner], algorithm=[none], cost=[not available])
               HiveProject($f0=[$0], $f1=[$1], $f2=[$2], $f3=[$3], $f4=[$4])
-                HiveAggregate(group=[{0, 1, 2}], agg#0=[sum($3)], agg#1=[count()])
-                  HiveProject($f0=[$1], $f1=[$2], $f2=[$3], $f3=[*(CAST($7):DECIMAL(10, 0), $8)])
-                    HiveJoin(condition=[=($6, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                      HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
-                        HiveTableScan(table=[[default, item]], table:alias=[item])
-                      HiveJoin(condition=[=($2, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                        HiveProject(ss_item_sk=[$0])
-                          HiveJoin(condition=[AND(=($1, $4), =($2, $5), =($3, $6))], joinType=[inner], algorithm=[none], cost=[not available])
-                            HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
-                              HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11))])
-                                HiveTableScan(table=[[default, item]], table:alias=[item])
-                            HiveProject($f0=[$0], $f1=[$1], $f2=[$2])
-                              HiveFilter(condition=[=($3, 3)])
-                                HiveAggregate(group=[{0, 1, 2}], agg#0=[count($3)])
-                                  HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3])
-                                    HiveUnion(all=[true])
-                                      HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3])
-                                        HiveAggregate(group=[{4, 5, 6}], agg#0=[count()])
-                                          HiveJoin(condition=[=($1, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                                            HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-                                              HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2])
-                                                HiveFilter(condition=[IS NOT NULL($0)])
-                                                  HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                                              HiveProject(d_date_sk=[$0])
-                                                HiveFilter(condition=[BETWEEN(false, $6, 1999, 2001)])
-                                                  HiveTableScan(table=[[default, date_dim]], table:alias=[d1])
-                                            HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
-                                              HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11))])
-                                                HiveTableScan(table=[[default, item]], table:alias=[iss])
-                                      HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3])
-                                        HiveAggregate(group=[{4, 5, 6}], agg#0=[count()])
-                                          HiveJoin(condition=[=($1, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                                            HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-                                              HiveProject(cs_sold_date_sk=[$0], cs_item_sk=[$15])
-                                                HiveFilter(condition=[IS NOT NULL($0)])
-                                                  HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
-                                              HiveProject(d_date_sk=[$0])
-                                                HiveFilter(condition=[BETWEEN(false, $6, 1999, 2001)])
-                                                  HiveTableScan(table=[[default, date_dim]], table:alias=[d2])
-                                            HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
-                                              HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11))])
-                                                HiveTableScan(table=[[default, item]], table:alias=[ics])
-                                      HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3])
-                                        HiveAggregate(group=[{4, 5, 6}], agg#0=[count()])
-                                          HiveJoin(condition=[=($1, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                                            HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-                                              HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$3])
-                                                HiveFilter(condition=[IS NOT NULL($0)])
-                                                  HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
-                                              HiveProject(d_date_sk=[$0])
-                                                HiveFilter(condition=[BETWEEN(false, $6, 1999, 2001)])
-                                                  HiveTableScan(table=[[default, date_dim]], table:alias=[d3])
-                                            HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
-                                              HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11))])
-                                                HiveTableScan(table=[[default, item]], table:alias=[iws])
-                        HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available])
-                          HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$3], ws_quantity=[$18], ws_list_price=[$20])
-                            HiveFilter(condition=[IS NOT NULL($0)])
-                              HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
-                          HiveProject(d_date_sk=[$0])
-                            HiveFilter(condition=[AND(=($6, 2000), =($8, 11))])
-                              HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+                HiveFilter(condition=[IS NOT NULL($3)])
+                  HiveAggregate(group=[{0, 1, 2}], agg#0=[sum($3)], agg#1=[count()])
+                    HiveProject($f0=[$1], $f1=[$2], $f2=[$3], $f3=[*(CAST($7):DECIMAL(10, 0), $8)])
+                      HiveJoin(condition=[=($6, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+                        HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
+                          HiveTableScan(table=[[default, item]], table:alias=[item])
+                        HiveJoin(condition=[=($2, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+                          HiveProject(ss_item_sk=[$0])
+                            HiveJoin(condition=[AND(=($1, $4), =($2, $5), =($3, $6))], joinType=[inner], algorithm=[none], cost=[not available])
+                              HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
+                                HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11))])
+                                  HiveTableScan(table=[[default, item]], table:alias=[item])
+                              HiveProject($f0=[$0], $f1=[$1], $f2=[$2])
+                                HiveFilter(condition=[=($3, 3)])
+                                  HiveAggregate(group=[{0, 1, 2}], agg#0=[count($3)])
+                                    HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3])
+                                      HiveUnion(all=[true])
+                                        HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3])
+                                          HiveAggregate(group=[{4, 5, 6}], agg#0=[count()])
+                                            HiveJoin(condition=[=($1, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                                              HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+                                                HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2])
+                                                  HiveFilter(condition=[IS NOT NULL($0)])
+                                                    HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                                                HiveProject(d_date_sk=[$0])
+                                                  HiveFilter(condition=[BETWEEN(false, $6, 1999, 2001)])
+                                                    HiveTableScan(table=[[default, date_dim]], table:alias=[d1])
+                                              HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
+                                                HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11))])
+                                                  HiveTableScan(table=[[default, item]], table:alias=[iss])
+                                        HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3])
+                                          HiveAggregate(group=[{4, 5, 6}], agg#0=[count()])
+                                            HiveJoin(condition=[=($1, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                                              HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+                                                HiveProject(cs_sold_date_sk=[$0], cs_item_sk=[$15])
+                                                  HiveFilter(condition=[IS NOT NULL($0)])
+                                                    HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
+                                                HiveProject(d_date_sk=[$0])
+                                                  HiveFilter(condition=[BETWEEN(false, $6, 1999, 2001)])
+                                                    HiveTableScan(table=[[default, date_dim]], table:alias=[d2])
+                                              HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
+                                                HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11))])
+                                                  HiveTableScan(table=[[default, item]], table:alias=[ics])
+                                        HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3])
+                                          HiveAggregate(group=[{4, 5, 6}], agg#0=[count()])
+                                            HiveJoin(condition=[=($1, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                                              HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+                                                HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$3])
+                                                  HiveFilter(condition=[IS NOT NULL($0)])
+                                                    HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
+                                                HiveProject(d_date_sk=[$0])
+                                                  HiveFilter(condition=[BETWEEN(false, $6, 1999, 2001)])
+                                                    HiveTableScan(table=[[default, date_dim]], table:alias=[d3])
+                                              HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11])
+                                                HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11))])
+                                                  HiveTableScan(table=[[default, item]], table:alias=[iws])
+                          HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available])
+                            HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$3], ws_quantity=[$18], ws_list_price=[$20])
+                              HiveFilter(condition=[IS NOT NULL($0)])
+                                HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
+                            HiveProject(d_date_sk=[$0])
+                              HiveFilter(condition=[AND(=($6, 2000), =($8, 11))])
+                                HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
               HiveProject($f0=[/($0, $1)])
-                HiveAggregate(group=[{}], agg#0=[sum($0)], agg#1=[count($0)])
-                  HiveProject($f0=[*(CAST($0):DECIMAL(10, 0), $1)])
-                    HiveUnion(all=[true])
-                      HiveProject(quantity=[$1], list_price=[$2])
-                        HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                          HiveProject(ss_sold_date_sk=[$0], ss_quantity=[$10], ss_list_price=[$12])
-                            HiveFilter(condition=[IS NOT NULL($0)])
-                              HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                          HiveProject(d_date_sk=[$0])
-                            HiveFilter(condition=[BETWEEN(false, $6, 1999, 2001)])
-                              HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-                      HiveProject(quantity=[$1], list_price=[$2])
-                        HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                          HiveProject(cs_sold_date_sk=[$0], cs_quantity=[$18], cs_list_price=[$20])
-                            HiveFilter(condition=[IS NOT NULL($0)])
-                              HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
-                          HiveProject(d_date_sk=[$0])
-                            HiveFilter(condition=[BETWEEN(false, $6, 1998, 2000)])
-                              HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-                      HiveProject(quantity=[$1], list_price=[$2])
-                        HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                          HiveProject(ws_sold_date_sk=[$0], ws_quantity=[$18], ws_list_price=[$20])
-                            HiveFilter(condition=[IS NOT NULL($0)])
-                              HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
-                          HiveProject(d_date_sk=[$0])
-                            HiveFilter(condition=[BETWEEN(false, $6, 1998, 2000)])
-                              HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+                HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($1))])
+                  HiveProject($f0=[$0], $f1=[$1])
+                    HiveAggregate(group=[{}], agg#0=[sum($0)], agg#1=[count($0)])
+                      HiveProject($f0=[*(CAST($0):DECIMAL(10, 0), $1)])
+                        HiveUnion(all=[true])
+                          HiveProject(quantity=[$1], list_price=[$2])
+                            HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                              HiveProject(ss_sold_date_sk=[$0], ss_quantity=[$10], ss_list_price=[$12])
+                                HiveFilter(condition=[IS NOT NULL($0)])
+                                  HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                              HiveProject(d_date_sk=[$0])
+                                HiveFilter(condition=[BETWEEN(false, $6, 1999, 2001)])
+                                  HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+                          HiveProject(quantity=[$1], list_price=[$2])
+                            HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                              HiveProject(cs_sold_date_sk=[$0], cs_quantity=[$18], cs_list_price=[$20])
+                                HiveFilter(condition=[IS NOT NULL($0)])
+                                  HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
+                              HiveProject(d_date_sk=[$0])
+                                HiveFilter(condition=[BETWEEN(false, $6, 1998, 2000)])
+                                  HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+                          HiveProject(quantity=[$1], list_price=[$2])
+                            HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                              HiveProject(ws_sold_date_sk=[$0], ws_quantity=[$18], ws_list_price=[$20])
+                                HiveFilter(condition=[IS NOT NULL($0)])
+                                  HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
+                              HiveProject(d_date_sk=[$0])
+                                HiveFilter(condition=[BETWEEN(false, $6, 1998, 2000)])
+                                  HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query23.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query23.q.out
index a348cc4..070894d 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query23.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query23.q.out
@@ -1,5 +1,5 @@
-Warning: Shuffle Join MERGEJOIN[352][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 17' is a cross product
-Warning: Shuffle Join MERGEJOIN[354][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 23' is a cross product
+Warning: Shuffle Join MERGEJOIN[358][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 17' is a cross product
+Warning: Shuffle Join MERGEJOIN[360][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 23' is a cross product
 PREHOOK: query: explain cbo
 with frequent_ss_items as 
  (select substr(i_item_desc,1,30) itemdesc,i_item_sk item_sk,d_date solddate,count(*) cnt
@@ -127,21 +127,24 @@ HiveAggregate(group=[{}], agg#0=[sum($0)])
               HiveAggregate(group=[{0}])
                 HiveJoin(condition=[>($1, $2)], joinType=[inner], algorithm=[none], cost=[not available])
                   HiveProject(ss_customer_sk=[$0], $f1=[$1])
-                    HiveAggregate(group=[{0}], agg#0=[sum($1)])
-                      HiveProject(ss_customer_sk=[CAST($3):INTEGER NOT NULL], *=[*(CAST($10):DECIMAL(10, 0), $13)])
-                        HiveFilter(condition=[IS NOT NULL($3)])
-                          HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                    HiveFilter(condition=[IS NOT NULL($1)])
+                      HiveAggregate(group=[{0}], agg#0=[sum($1)])
+                        HiveProject(ss_customer_sk=[CAST($3):INTEGER NOT NULL], *=[*(CAST($10):DECIMAL(10, 0), $13)])
+                          HiveFilter(condition=[IS NOT NULL($3)])
+                            HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
                   HiveProject(*=[*(0.95:DECIMAL(3, 2), $0)])
-                    HiveAggregate(group=[{}], agg#0=[max($1)])
-                      HiveProject(ss_customer_sk=[$0], $f1=[$1])
-                        HiveAggregate(group=[{1}], agg#0=[sum($2)])
-                          HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                            HiveProject(ss_sold_date_sk=[$0], ss_customer_sk=[CAST($3):INTEGER NOT NULL], *=[*(CAST($10):DECIMAL(10, 0), $13)])
-                              HiveFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($0))])
-                                HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                            HiveProject(d_date_sk=[$0])
-                              HiveFilter(condition=[IN($6, 1999, 2000, 2001, 2002)])
-                                HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+                    HiveFilter(condition=[IS NOT NULL($0)])
+                      HiveProject($f0=[$0])
+                        HiveAggregate(group=[{}], agg#0=[max($1)])
+                          HiveProject(ss_customer_sk=[$0], $f1=[$1])
+                            HiveAggregate(group=[{1}], agg#0=[sum($2)])
+                              HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                                HiveProject(ss_sold_date_sk=[$0], ss_customer_sk=[CAST($3):INTEGER NOT NULL], *=[*(CAST($10):DECIMAL(10, 0), $13)])
+                                  HiveFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($0))])
+                                    HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                                HiveProject(d_date_sk=[$0])
+                                  HiveFilter(condition=[IN($6, 1999, 2000, 2001, 2002)])
+                                    HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
             HiveJoin(condition=[=($0, $5)], joinType=[inner], algorithm=[none], cost=[not available])
               HiveProject(cs_sold_date_sk=[$0], cs_bill_customer_sk=[$3], cs_item_sk=[$15], cs_quantity=[$18], cs_list_price=[$20])
                 HiveFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($0))])
@@ -170,21 +173,24 @@ HiveAggregate(group=[{}], agg#0=[sum($0)])
               HiveAggregate(group=[{0}])
                 HiveJoin(condition=[>($1, $2)], joinType=[inner], algorithm=[none], cost=[not available])
                   HiveProject(ss_customer_sk=[$0], $f1=[$1])
-                    HiveAggregate(group=[{0}], agg#0=[sum($1)])
-                      HiveProject(ss_customer_sk=[CAST($3):INTEGER NOT NULL], *=[*(CAST($10):DECIMAL(10, 0), $13)])
-                        HiveFilter(condition=[IS NOT NULL($3)])
-                          HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                    HiveFilter(condition=[IS NOT NULL($1)])
+                      HiveAggregate(group=[{0}], agg#0=[sum($1)])
+                        HiveProject(ss_customer_sk=[CAST($3):INTEGER NOT NULL], *=[*(CAST($10):DECIMAL(10, 0), $13)])
+                          HiveFilter(condition=[IS NOT NULL($3)])
+                            HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
                   HiveProject(*=[*(0.95:DECIMAL(3, 2), $0)])
-                    HiveAggregate(group=[{}], agg#0=[max($1)])
-                      HiveProject(ss_customer_sk=[$0], $f1=[$1])
-                        HiveAggregate(group=[{1}], agg#0=[sum($2)])
-                          HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                            HiveProject(ss_sold_date_sk=[$0], ss_customer_sk=[CAST($3):INTEGER NOT NULL], *=[*(CAST($10):DECIMAL(10, 0), $13)])
-                              HiveFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($0))])
-                                HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                            HiveProject(d_date_sk=[$0])
-                              HiveFilter(condition=[IN($6, 1999, 2000, 2001, 2002)])
-                                HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+                    HiveFilter(condition=[IS NOT NULL($0)])
+                      HiveProject($f0=[$0])
+                        HiveAggregate(group=[{}], agg#0=[max($1)])
+                          HiveProject(ss_customer_sk=[$0], $f1=[$1])
+                            HiveAggregate(group=[{1}], agg#0=[sum($2)])
+                              HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                                HiveProject(ss_sold_date_sk=[$0], ss_customer_sk=[CAST($3):INTEGER NOT NULL], *=[*(CAST($10):DECIMAL(10, 0), $13)])
+                                  HiveFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($0))])
+                                    HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                                HiveProject(d_date_sk=[$0])
+                                  HiveFilter(condition=[IN($6, 1999, 2000, 2001, 2002)])
+                                    HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
             HiveJoin(condition=[=($0, $5)], joinType=[inner], algorithm=[none], cost=[not available])
               HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$3], ws_bill_customer_sk=[$4], ws_quantity=[$18], ws_list_price=[$20])
                 HiveFilter(condition=[AND(IS NOT NULL($4), IS NOT NULL($0))])
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query24.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query24.q.out
index d32e0c8..9d93d73 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query24.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query24.q.out
@@ -1,4 +1,4 @@
-Warning: Shuffle Join MERGEJOIN[297][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 6' is a cross product
+Warning: Shuffle Join MERGEJOIN[301][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 6' is a cross product
 PREHOOK: query: explain cbo
 with ssales as
 (select c_last_name
@@ -116,57 +116,61 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 CBO PLAN:
 HiveProject($f0=[$0], $f1=[$1], $f2=[$2], $f3=[$3])
   HiveJoin(condition=[>($3, $4)], joinType=[inner], algorithm=[none], cost=[not available])
-    HiveProject(c_last_name=[$1], c_first_name=[$0], s_store_name=[$2], $f3=[$3])
-      HiveAggregate(group=[{0, 1, 3}], agg#0=[sum($9)])
-        HiveProject(c_first_name=[$0], c_last_name=[$1], ca_state=[$2], s_store_name=[$3], s_state=[$4], i_current_price=[$5], i_size=[$6], i_units=[$7], i_manager_id=[$8], $f9=[$9])
-          HiveAggregate(group=[{9, 10, 13, 17, 18, 21, 22, 23, 24}], agg#0=[sum($4)])
-            HiveJoin(condition=[=($0, $20)], joinType=[inner], algorithm=[none], cost=[not available])
-              HiveJoin(condition=[AND(=($1, $7), =($2, $16))], joinType=[inner], algorithm=[none], cost=[not available])
-                HiveJoin(condition=[AND(=($3, $6), =($0, $5))], joinType=[inner], algorithm=[none], cost=[not available])
-                  HiveProject(ss_item_sk=[$2], ss_customer_sk=[$3], ss_store_sk=[$7], ss_ticket_number=[$9], ss_sales_price=[$13])
-                    HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($3))])
-                      HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                  HiveProject(sr_item_sk=[$2], sr_ticket_number=[$9])
-                    HiveTableScan(table=[[default, store_returns]], table:alias=[store_returns])
-                HiveProject(c_customer_sk=[$0], c_current_addr_sk=[$1], c_first_name=[$2], c_last_name=[$3], c_birth_country=[$4], ca_address_sk=[$5], ca_state=[$6], ca_zip=[$7], UPPER=[$8], s_store_sk=[$9], s_store_name=[$10], s_state=[$11], s_zip=[$12])
-                  HiveJoin(condition=[AND(=($1, $5), <>($4, $8))], joinType=[inner], algorithm=[none], cost=[not available])
-                    HiveProject(c_customer_sk=[$0], c_current_addr_sk=[$4], c_first_name=[$8], c_last_name=[$9], c_birth_country=[$14])
-                      HiveFilter(condition=[IS NOT NULL($4)])
-                        HiveTableScan(table=[[default, customer]], table:alias=[customer])
-                    HiveJoin(condition=[=($7, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-                      HiveProject(ca_address_sk=[$0], ca_state=[$8], ca_zip=[$9], UPPER=[UPPER($10)])
-                        HiveFilter(condition=[IS NOT NULL($9)])
-                          HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
-                      HiveProject(s_store_sk=[$0], s_store_name=[$5], s_state=[$24], s_zip=[$25])
-                        HiveFilter(condition=[AND(=($10, 7), IS NOT NULL($25))])
-                          HiveTableScan(table=[[default, store]], table:alias=[store])
-              HiveProject(i_item_sk=[$0], i_current_price=[$5], i_size=[$15], i_units=[$18], i_manager_id=[$20])
-                HiveFilter(condition=[=($17, _UTF-16LE'orchid')])
-                  HiveTableScan(table=[[default, item]], table:alias=[item])
+    HiveProject(c_last_name=[$0], c_first_name=[$1], s_store_name=[$2], $f3=[$3])
+      HiveFilter(condition=[IS NOT NULL($3)])
+        HiveProject(c_last_name=[$1], c_first_name=[$0], s_store_name=[$2], $f3=[$3])
+          HiveAggregate(group=[{0, 1, 3}], agg#0=[sum($9)])
+            HiveProject(c_first_name=[$0], c_last_name=[$1], ca_state=[$2], s_store_name=[$3], s_state=[$4], i_current_price=[$5], i_size=[$6], i_units=[$7], i_manager_id=[$8], $f9=[$9])
+              HiveAggregate(group=[{9, 10, 13, 17, 18, 21, 22, 23, 24}], agg#0=[sum($4)])
+                HiveJoin(condition=[=($0, $20)], joinType=[inner], algorithm=[none], cost=[not available])
+                  HiveJoin(condition=[AND(=($1, $7), =($2, $16))], joinType=[inner], algorithm=[none], cost=[not available])
+                    HiveJoin(condition=[AND(=($3, $6), =($0, $5))], joinType=[inner], algorithm=[none], cost=[not available])
+                      HiveProject(ss_item_sk=[$2], ss_customer_sk=[$3], ss_store_sk=[$7], ss_ticket_number=[$9], ss_sales_price=[$13])
+                        HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($3))])
+                          HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                      HiveProject(sr_item_sk=[$2], sr_ticket_number=[$9])
+                        HiveTableScan(table=[[default, store_returns]], table:alias=[store_returns])
+                    HiveProject(c_customer_sk=[$0], c_current_addr_sk=[$1], c_first_name=[$2], c_last_name=[$3], c_birth_country=[$4], ca_address_sk=[$5], ca_state=[$6], ca_zip=[$7], UPPER=[$8], s_store_sk=[$9], s_store_name=[$10], s_state=[$11], s_zip=[$12])
+                      HiveJoin(condition=[AND(=($1, $5), <>($4, $8))], joinType=[inner], algorithm=[none], cost=[not available])
+                        HiveProject(c_customer_sk=[$0], c_current_addr_sk=[$4], c_first_name=[$8], c_last_name=[$9], c_birth_country=[$14])
+                          HiveFilter(condition=[IS NOT NULL($4)])
+                            HiveTableScan(table=[[default, customer]], table:alias=[customer])
+                        HiveJoin(condition=[=($7, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+                          HiveProject(ca_address_sk=[$0], ca_state=[$8], ca_zip=[$9], UPPER=[UPPER($10)])
+                            HiveFilter(condition=[IS NOT NULL($9)])
+                              HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
+                          HiveProject(s_store_sk=[$0], s_store_name=[$5], s_state=[$24], s_zip=[$25])
+                            HiveFilter(condition=[AND(=($10, 7), IS NOT NULL($25))])
+                              HiveTableScan(table=[[default, store]], table:alias=[store])
+                  HiveProject(i_item_sk=[$0], i_current_price=[$5], i_size=[$15], i_units=[$18], i_manager_id=[$20])
+                    HiveFilter(condition=[=($17, _UTF-16LE'orchid')])
+                      HiveTableScan(table=[[default, item]], table:alias=[item])
     HiveProject(_o__c0=[*(0.05:DECIMAL(3, 2), /($0, $1))])
-      HiveAggregate(group=[{}], agg#0=[sum($10)], agg#1=[count($10)])
-        HiveProject(c_first_name=[$0], c_last_name=[$1], ca_state=[$2], s_store_name=[$3], s_state=[$4], i_current_price=[$5], i_size=[$6], i_color=[$7], i_units=[$8], i_manager_id=[$9], $f10=[$10])
-          HiveAggregate(group=[{9, 10, 13, 17, 18, 21, 22, 23, 24, 25}], agg#0=[sum($4)])
-            HiveJoin(condition=[=($0, $20)], joinType=[inner], algorithm=[none], cost=[not available])
-              HiveJoin(condition=[AND(=($1, $7), =($2, $16))], joinType=[inner], algorithm=[none], cost=[not available])
-                HiveJoin(condition=[AND(=($3, $6), =($0, $5))], joinType=[inner], algorithm=[none], cost=[not available])
-                  HiveProject(ss_item_sk=[$2], ss_customer_sk=[$3], ss_store_sk=[$7], ss_ticket_number=[$9], ss_sales_price=[$13])
-                    HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($3))])
-                      HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                  HiveProject(sr_item_sk=[$2], sr_ticket_number=[$9])
-                    HiveTableScan(table=[[default, store_returns]], table:alias=[store_returns])
-                HiveProject(c_customer_sk=[$0], c_current_addr_sk=[$1], c_first_name=[$2], c_last_name=[$3], c_birth_country=[$4], ca_address_sk=[$5], ca_state=[$6], ca_zip=[$7], UPPER=[$8], s_store_sk=[$9], s_store_name=[$10], s_state=[$11], s_zip=[$12])
-                  HiveJoin(condition=[AND(=($1, $5), <>($4, $8))], joinType=[inner], algorithm=[none], cost=[not available])
-                    HiveProject(c_customer_sk=[$0], c_current_addr_sk=[$4], c_first_name=[$8], c_last_name=[$9], c_birth_country=[$14])
-                      HiveFilter(condition=[IS NOT NULL($4)])
-                        HiveTableScan(table=[[default, customer]], table:alias=[customer])
-                    HiveJoin(condition=[=($7, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-                      HiveProject(ca_address_sk=[$0], ca_state=[$8], ca_zip=[$9], UPPER=[UPPER($10)])
-                        HiveFilter(condition=[IS NOT NULL($9)])
-                          HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
-                      HiveProject(s_store_sk=[$0], s_store_name=[$5], s_state=[$24], s_zip=[$25])
-                        HiveFilter(condition=[AND(=($10, 7), IS NOT NULL($25))])
-                          HiveTableScan(table=[[default, store]], table:alias=[store])
-              HiveProject(i_item_sk=[$0], i_current_price=[$5], i_size=[$15], i_color=[$17], i_units=[$18], i_manager_id=[$20])
-                HiveTableScan(table=[[default, item]], table:alias=[item])
+      HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($1))])
+        HiveProject($f0=[$0], $f1=[$1])
+          HiveAggregate(group=[{}], agg#0=[sum($10)], agg#1=[count($10)])
+            HiveProject(c_first_name=[$0], c_last_name=[$1], ca_state=[$2], s_store_name=[$3], s_state=[$4], i_current_price=[$5], i_size=[$6], i_color=[$7], i_units=[$8], i_manager_id=[$9], $f10=[$10])
+              HiveAggregate(group=[{9, 10, 13, 17, 18, 21, 22, 23, 24, 25}], agg#0=[sum($4)])
+                HiveJoin(condition=[=($0, $20)], joinType=[inner], algorithm=[none], cost=[not available])
+                  HiveJoin(condition=[AND(=($1, $7), =($2, $16))], joinType=[inner], algorithm=[none], cost=[not available])
+                    HiveJoin(condition=[AND(=($3, $6), =($0, $5))], joinType=[inner], algorithm=[none], cost=[not available])
+                      HiveProject(ss_item_sk=[$2], ss_customer_sk=[$3], ss_store_sk=[$7], ss_ticket_number=[$9], ss_sales_price=[$13])
+                        HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($3))])
+                          HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                      HiveProject(sr_item_sk=[$2], sr_ticket_number=[$9])
+                        HiveTableScan(table=[[default, store_returns]], table:alias=[store_returns])
+                    HiveProject(c_customer_sk=[$0], c_current_addr_sk=[$1], c_first_name=[$2], c_last_name=[$3], c_birth_country=[$4], ca_address_sk=[$5], ca_state=[$6], ca_zip=[$7], UPPER=[$8], s_store_sk=[$9], s_store_name=[$10], s_state=[$11], s_zip=[$12])
+                      HiveJoin(condition=[AND(=($1, $5), <>($4, $8))], joinType=[inner], algorithm=[none], cost=[not available])
+                        HiveProject(c_customer_sk=[$0], c_current_addr_sk=[$4], c_first_name=[$8], c_last_name=[$9], c_birth_country=[$14])
+                          HiveFilter(condition=[IS NOT NULL($4)])
+                            HiveTableScan(table=[[default, customer]], table:alias=[customer])
+                        HiveJoin(condition=[=($7, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+                          HiveProject(ca_address_sk=[$0], ca_state=[$8], ca_zip=[$9], UPPER=[UPPER($10)])
+                            HiveFilter(condition=[IS NOT NULL($9)])
+                              HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
+                          HiveProject(s_store_sk=[$0], s_store_name=[$5], s_state=[$24], s_zip=[$25])
+                            HiveFilter(condition=[AND(=($10, 7), IS NOT NULL($25))])
+                              HiveTableScan(table=[[default, store]], table:alias=[store])
+                  HiveProject(i_item_sk=[$0], i_current_price=[$5], i_size=[$15], i_color=[$17], i_units=[$18], i_manager_id=[$20])
+                    HiveTableScan(table=[[default, item]], table:alias=[item])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query30.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query30.q.out
index 7c8bfeb..564fe6a 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query30.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query30.q.out
@@ -81,22 +81,9 @@ HiveSortLimit(sort0=[$0], sort1=[$1], sort2=[$2], sort3=[$3], sort4=[$4], sort5=
             HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
       HiveProject(wr_returning_customer_sk=[$0], ca_state=[$1], $f2=[$2], _o__c0=[$3], ctr_state=[$4])
         HiveJoin(condition=[AND(=($1, $4), >($2, $3))], joinType=[inner], algorithm=[none], cost=[not available])
-          HiveProject(wr_returning_customer_sk=[$1], ca_state=[$0], $f2=[$2])
-            HiveAggregate(group=[{1, 3}], agg#0=[sum($5)])
-              HiveJoin(condition=[=($4, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                HiveProject(ca_address_sk=[$0], ca_state=[$8])
-                  HiveFilter(condition=[IS NOT NULL($8)])
-                    HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
-                HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available])
-                  HiveProject(wr_returned_date_sk=[$0], wr_returning_customer_sk=[$7], wr_returning_addr_sk=[$10], wr_return_amt=[$15])
-                    HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($10), IS NOT NULL($7))])
-                      HiveTableScan(table=[[default, web_returns]], table:alias=[web_returns])
-                  HiveProject(d_date_sk=[$0])
-                    HiveFilter(condition=[=($6, 2002)])
-                      HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-          HiveProject(_o__c0=[*(/($1, $2), 1.2:DECIMAL(2, 1))], ctr_state=[$0])
-            HiveAggregate(group=[{0}], agg#0=[sum($2)], agg#1=[count($2)])
-              HiveProject(ca_state=[$0], wr_returning_customer_sk=[$1], $f2=[$2])
+          HiveProject(wr_returning_customer_sk=[$0], ca_state=[$1], $f2=[$2])
+            HiveFilter(condition=[IS NOT NULL($2)])
+              HiveProject(wr_returning_customer_sk=[$1], ca_state=[$0], $f2=[$2])
                 HiveAggregate(group=[{1, 3}], agg#0=[sum($5)])
                   HiveJoin(condition=[=($4, $0)], joinType=[inner], algorithm=[none], cost=[not available])
                     HiveProject(ca_address_sk=[$0], ca_state=[$8])
@@ -104,9 +91,25 @@ HiveSortLimit(sort0=[$0], sort1=[$1], sort2=[$2], sort3=[$3], sort4=[$4], sort5=
                         HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
                     HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available])
                       HiveProject(wr_returned_date_sk=[$0], wr_returning_customer_sk=[$7], wr_returning_addr_sk=[$10], wr_return_amt=[$15])
-                        HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($10))])
+                        HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($10), IS NOT NULL($7))])
                           HiveTableScan(table=[[default, web_returns]], table:alias=[web_returns])
                       HiveProject(d_date_sk=[$0])
                         HiveFilter(condition=[=($6, 2002)])
                           HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+          HiveProject(_o__c0=[*(/($1, $2), 1.2:DECIMAL(2, 1))], ctr_state=[$0])
+            HiveFilter(condition=[AND(IS NOT NULL($1), IS NOT NULL($2))])
+              HiveAggregate(group=[{0}], agg#0=[sum($2)], agg#1=[count($2)])
+                HiveProject(ca_state=[$0], wr_returning_customer_sk=[$1], $f2=[$2])
+                  HiveAggregate(group=[{1, 3}], agg#0=[sum($5)])
+                    HiveJoin(condition=[=($4, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+                      HiveProject(ca_address_sk=[$0], ca_state=[$8])
+                        HiveFilter(condition=[IS NOT NULL($8)])
+                          HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
+                      HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available])
+                        HiveProject(wr_returned_date_sk=[$0], wr_returning_customer_sk=[$7], wr_returning_addr_sk=[$10], wr_return_amt=[$15])
+                          HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($10))])
+                            HiveTableScan(table=[[default, web_returns]], table:alias=[web_returns])
+                        HiveProject(d_date_sk=[$0])
+                          HiveFilter(condition=[=($6, 2002)])
+                            HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query32.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query32.q.out
index ba9ac46..5ba034c 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query32.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query32.q.out
@@ -62,26 +62,26 @@ POSTHOOK: Input: default@item
 POSTHOOK: Output: hdfs://### HDFS PATH ###
 CBO PLAN:
 HiveAggregate(group=[{}], agg#0=[sum($2)])
-  HiveJoin(condition=[AND(>($2, $5), =($6, $1))], joinType=[inner], algorithm=[none], cost=[not available])
-    HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-      HiveProject(cs_sold_date_sk=[$0], cs_item_sk=[$15], cs_ext_discount_amt=[$22])
-        HiveFilter(condition=[IS NOT NULL($0)])
-          HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
-      HiveProject(d_date_sk=[$0])
-        HiveFilter(condition=[BETWEEN(false, CAST($2):TIMESTAMP(9), 1998-03-18 00:00:00:TIMESTAMP(9), 1998-06-16 00:00:00:TIMESTAMP(9))])
-          HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-    HiveProject(cs_item_sk=[$0], CAST3=[$1], i_item_sk=[$2])
-      HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-        HiveProject(cs_item_sk=[$0], CAST3=[CAST(*(1.3:DECIMAL(2, 1), /($1, $2))):DECIMAL(14, 7)])
-          HiveAggregate(group=[{1}], agg#0=[sum($2)], agg#1=[count($2)])
-            HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-              HiveProject(cs_sold_date_sk=[$0], cs_item_sk=[$15], cs_ext_discount_amt=[$22])
-                HiveFilter(condition=[IS NOT NULL($0)])
-                  HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
-              HiveProject(d_date_sk=[$0])
-                HiveFilter(condition=[BETWEEN(false, CAST($2):TIMESTAMP(9), 1998-03-18 00:00:00:TIMESTAMP(9), 1998-06-16 00:00:00:TIMESTAMP(9))])
-                  HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-        HiveProject(i_item_sk=[$0])
-          HiveFilter(condition=[=($13, 269)])
-            HiveTableScan(table=[[default, item]], table:alias=[item])
+  HiveJoin(condition=[AND(=($5, $4), >($2, $6))], joinType=[inner], algorithm=[none], cost=[not available])
+    HiveJoin(condition=[=($4, $1)], joinType=[inner], algorithm=[none], cost=[not available])
+      HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+        HiveProject(cs_sold_date_sk=[$0], cs_item_sk=[$15], cs_ext_discount_amt=[$22])
+          HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($22))])
+            HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
+        HiveProject(d_date_sk=[$0])
+          HiveFilter(condition=[BETWEEN(false, CAST($2):TIMESTAMP(9), 1998-03-18 00:00:00:TIMESTAMP(9), 1998-06-16 00:00:00:TIMESTAMP(9))])
+            HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+      HiveProject(i_item_sk=[$0])
+        HiveFilter(condition=[=($13, 269)])
+          HiveTableScan(table=[[default, item]], table:alias=[item])
+    HiveProject(cs_item_sk=[$0], CAST3=[CAST(*(1.3:DECIMAL(2, 1), /($1, $2))):DECIMAL(14, 7)])
+      HiveFilter(condition=[IS NOT NULL(CAST(*(1.3:DECIMAL(2, 1), /($1, $2))):DECIMAL(14, 7))])
+        HiveAggregate(group=[{1}], agg#0=[sum($2)], agg#1=[count($2)])
+          HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+            HiveProject(cs_sold_date_sk=[$0], cs_item_sk=[$15], cs_ext_discount_amt=[$22])
+              HiveFilter(condition=[IS NOT NULL($0)])
+                HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
+            HiveProject(d_date_sk=[$0])
+              HiveFilter(condition=[BETWEEN(false, CAST($2):TIMESTAMP(9), 1998-03-18 00:00:00:TIMESTAMP(9), 1998-06-16 00:00:00:TIMESTAMP(9))])
+                HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query54.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query54.q.out
index f123086..cdeab44 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query54.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query54.q.out
@@ -1,7 +1,7 @@
-Warning: Shuffle Join MERGEJOIN[274][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
-Warning: Shuffle Join MERGEJOIN[280][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
-Warning: Shuffle Join MERGEJOIN[279][tables = [$hdt$_4, $hdt$_5]] in Stage 'Reducer 15' is a cross product
-Warning: Shuffle Join MERGEJOIN[282][tables = [$hdt$_4, $hdt$_5, $hdt$_6]] in Stage 'Reducer 16' is a cross product
+Warning: Shuffle Join MERGEJOIN[280][tables = [$hdt$_4, $hdt$_5]] in Stage 'Reducer 15' is a cross product
+Warning: Shuffle Join MERGEJOIN[283][tables = [$hdt$_4, $hdt$_5, $hdt$_6]] in Stage 'Reducer 16' is a cross product
+Warning: Shuffle Join MERGEJOIN[275][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+Warning: Shuffle Join MERGEJOIN[281][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: explain cbo
 with my_customers as (
  select distinct c_customer_sk
@@ -196,15 +196,16 @@ HiveSortLimit(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC], fetch=[100])
               HiveJoin(condition=[<=($1, $3)], joinType=[inner], algorithm=[none], cost=[not available])
                 HiveJoin(condition=[<=($2, $1)], joinType=[inner], algorithm=[none], cost=[not available])
                   HiveProject(d_date_sk=[$0], d_month_seq=[$3])
-                    HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+                    HiveFilter(condition=[IS NOT NULL($3)])
+                      HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
                   HiveProject($f0=[$0])
                     HiveAggregate(group=[{0}])
                       HiveProject($f0=[+($3, 1)])
-                        HiveFilter(condition=[AND(=($6, 1999), =($8, 3))])
+                        HiveFilter(condition=[AND(=($6, 1999), =($8, 3), IS NOT NULL($3))])
                           HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
                 HiveProject($f0=[$0])
                   HiveAggregate(group=[{0}])
                     HiveProject($f0=[+($3, 3)])
-                      HiveFilter(condition=[AND(=($6, 1999), =($8, 3))])
+                      HiveFilter(condition=[AND(=($6, 1999), =($8, 3), IS NOT NULL($3))])
                         HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query6.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query6.q.out
index 4c0cb9c..00aafff 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query6.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query6.q.out
@@ -1,4 +1,4 @@
-Warning: Map Join MAPJOIN[168][bigTable=?] in task 'Map 1' is a cross product
+Warning: Map Join MAPJOIN[169][bigTable=?] in task 'Map 1' is a cross product
 PREHOOK: query: explain cbo
 select  a.ca_state state, count(*) cnt
  from customer_address a
@@ -66,7 +66,7 @@ HiveSortLimit(sort0=[$1], dir0=[ASC], fetch=[100])
   HiveProject(ca_state=[$0], $f1=[$1])
     HiveFilter(condition=[>=($1, 10)])
       HiveAggregate(group=[{4}], agg#0=[count()])
-        HiveJoin(condition=[=($6, $11)], joinType=[inner], algorithm=[none], cost=[not available])
+        HiveJoin(condition=[=($5, $13)], joinType=[inner], algorithm=[none], cost=[not available])
           HiveJoin(condition=[=($0, $7)], joinType=[inner], algorithm=[none], cost=[not available])
             HiveJoin(condition=[=($3, $1)], joinType=[inner], algorithm=[none], cost=[not available])
               HiveJoin(condition=[true], joinType=[inner], algorithm=[none], cost=[not available])
@@ -83,26 +83,27 @@ HiveSortLimit(sort0=[$1], dir0=[ASC], fetch=[100])
                               HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
               HiveProject(ca_address_sk=[$0], ca_state=[$8])
                 HiveTableScan(table=[[default, customer_address]], table:alias=[a])
-            HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$1], ss_customer_sk=[$2], d_date_sk=[$3], d_month_seq=[$4], d_month_seq0=[$5])
-              HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+            HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$1], ss_customer_sk=[$2], i_item_sk=[$3], i_current_price=[$4], i_category=[$5], i_category0=[$6], *=[$7])
+              HiveJoin(condition=[=($1, $3)], joinType=[inner], algorithm=[none], cost=[not available])
                 HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2], ss_customer_sk=[$3])
                   HiveFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($0))])
                     HiveTableScan(table=[[default, store_sales]], table:alias=[s])
-                HiveJoin(condition=[=($1, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-                  HiveProject(d_date_sk=[$0], d_month_seq=[$3])
-                    HiveFilter(condition=[IS NOT NULL($3)])
-                      HiveTableScan(table=[[default, date_dim]], table:alias=[d])
-                  HiveProject(d_month_seq=[$0])
-                    HiveAggregate(group=[{3}])
-                      HiveFilter(condition=[AND(=($6, 2000), =($8, 2), IS NOT NULL($3))])
-                        HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-          HiveProject(i_item_sk=[$0], i_current_price=[$1], i_category=[$2], i_category0=[$3], *=[$4])
-            HiveJoin(condition=[AND(=($3, $2), >($1, $4))], joinType=[inner], algorithm=[none], cost=[not available])
-              HiveProject(i_item_sk=[$0], i_current_price=[$5], i_category=[$12])
-                HiveFilter(condition=[IS NOT NULL($12)])
-                  HiveTableScan(table=[[default, item]], table:alias=[i])
-              HiveProject(i_category=[$0], *=[*(1.2:DECIMAL(2, 1), CAST(/($1, $2)):DECIMAL(16, 6))])
-                HiveAggregate(group=[{12}], agg#0=[sum($5)], agg#1=[count($5)])
-                  HiveFilter(condition=[IS NOT NULL($12)])
-                    HiveTableScan(table=[[default, item]], table:alias=[j])
+                HiveJoin(condition=[AND(=($3, $2), >($1, $4))], joinType=[inner], algorithm=[none], cost=[not available])
+                  HiveProject(i_item_sk=[$0], i_current_price=[$5], i_category=[$12])
+                    HiveFilter(condition=[AND(IS NOT NULL($12), IS NOT NULL($5))])
+                      HiveTableScan(table=[[default, item]], table:alias=[i])
+                  HiveProject(i_category=[$0], *=[*(1.2:DECIMAL(2, 1), CAST(/($1, $2)):DECIMAL(16, 6))])
+                    HiveFilter(condition=[IS NOT NULL(CAST(/($1, $2)):DECIMAL(16, 6))])
+                      HiveAggregate(group=[{12}], agg#0=[sum($5)], agg#1=[count($5)])
+                        HiveFilter(condition=[IS NOT NULL($12)])
+                          HiveTableScan(table=[[default, item]], table:alias=[j])
+          HiveProject(d_date_sk=[$0], d_month_seq=[$1], d_month_seq0=[$2])
+            HiveJoin(condition=[=($1, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+              HiveProject(d_date_sk=[$0], d_month_seq=[$3])
+                HiveFilter(condition=[IS NOT NULL($3)])
+                  HiveTableScan(table=[[default, date_dim]], table:alias=[d])
+              HiveProject(d_month_seq=[$0])
+                HiveAggregate(group=[{3}])
+                  HiveFilter(condition=[AND(=($6, 2000), =($8, 2), IS NOT NULL($3))])
+                    HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query64.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query64.q.out
index 10f1a81..b66c32f 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query64.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query64.q.out
@@ -265,124 +265,128 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 CBO PLAN:
 HiveProject(product_name=[$0], store_name=[$1], store_zip=[$2], b_street_number=[$3], b_streen_name=[$4], b_city=[$5], b_zip=[$6], c_street_number=[$7], c_street_name=[$8], c_city=[$9], c_zip=[$10], syear=[CAST(2000):INTEGER], cnt=[$11], s1=[$12], s2=[$13], s3=[$14], s11=[$15], s21=[$16], s31=[$17], syear1=[CAST(2001):INTEGER], cnt1=[$18])
   HiveSortLimit(sort0=[$0], sort1=[$1], sort2=[$18], dir0=[ASC], dir1=[ASC], dir2=[ASC])
-    HiveProject(product_name=[$0], store_name=[$2], store_zip=[$3], b_street_number=[$4], b_streen_name=[$5], b_city=[$6], b_zip=[$7], c_street_number=[$8], c_street_name=[$9], c_city=[$10], c_zip=[$11], cnt=[$12], s1=[$13], s2=[$14], s3=[$15], s11=[$20], s21=[$21], s31=[$22], cnt1=[$19])
-      HiveJoin(condition=[AND(=($1, $16), <=($19, $12), =($2, $17), =($3, $18))], joinType=[inner], algorithm=[none], cost=[not available])
-        HiveProject($f0=[$1], $f1=[$0], $f2=[$2], $f3=[$3], $f4=[$6], $f5=[$7], $f6=[$8], $f7=[$9], $f8=[$10], $f9=[$11], $f10=[$12], $f11=[$13], $f15=[$14], $f16=[$15], $f17=[$16], $f18=[$17])
-          HiveAggregate(group=[{19, 20, 24, 25, 29, 31, 37, 38, 39, 40, 42, 43, 44, 45}], agg#0=[count()], agg#1=[sum($16)], agg#2=[sum($17)], agg#3=[sum($18)])
-            HiveJoin(condition=[=($5, $41)], joinType=[inner], algorithm=[none], cost=[not available])
-              HiveJoin(condition=[=($13, $36)], joinType=[inner], algorithm=[none], cost=[not available])
-                HiveJoin(condition=[AND(=($3, $34), <>($33, $35))], joinType=[inner], algorithm=[none], cost=[not available])
-                  HiveJoin(condition=[=($11, $32)], joinType=[inner], algorithm=[none], cost=[not available])
-                    HiveJoin(condition=[=($6, $30)], joinType=[inner], algorithm=[none], cost=[not available])
-                      HiveJoin(condition=[=($7, $28)], joinType=[inner], algorithm=[none], cost=[not available])
-                        HiveJoin(condition=[=($4, $27)], joinType=[inner], algorithm=[none], cost=[not available])
-                          HiveJoin(condition=[=($12, $26)], joinType=[inner], algorithm=[none], cost=[not available])
-                            HiveJoin(condition=[=($14, $23)], joinType=[inner], algorithm=[none], cost=[not available])
-                              HiveJoin(condition=[AND(=($9, $0), =($15, $1))], joinType=[inner], algorithm=[none], cost=[not available])
-                                HiveProject(sr_item_sk=[$2], sr_ticket_number=[$9])
-                                  HiveTableScan(table=[[default, store_returns]], table:alias=[store_returns])
-                                HiveJoin(condition=[=($7, $20)], joinType=[inner], algorithm=[none], cost=[not available])
-                                  HiveJoin(condition=[=($8, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                                    HiveProject(c_customer_sk=[$0], c_current_cdemo_sk=[$2], c_current_hdemo_sk=[$3], c_current_addr_sk=[$4], c_first_shipto_date_sk=[$5], c_first_sales_date_sk=[$6])
-                                      HiveFilter(condition=[AND(IS NOT NULL($6), IS NOT NULL($5), IS NOT NULL($2), IS NOT NULL($3), IS NOT NULL($4))])
-                                        HiveTableScan(table=[[default, customer]], table:alias=[customer])
-                                    HiveJoin(condition=[=($0, $13)], joinType=[inner], algorithm=[none], cost=[not available])
-                                      HiveJoin(condition=[=($1, $11)], joinType=[inner], algorithm=[none], cost=[not available])
-                                        HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2], ss_customer_sk=[$3], ss_cdemo_sk=[$4], ss_hdemo_sk=[$5], ss_addr_sk=[$6], ss_store_sk=[$7], ss_ticket_number=[$9], ss_wholesale_cost=[$11], ss_list_price=[$12], ss_coupon_amt=[$19])
-                                          HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7), IS NOT NULL($3), IS NOT NULL($4), IS NOT NULL($8), IS NOT NULL($5), IS NOT NULL($6))])
-                                            HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                                        HiveProject(i_item_sk=[$0], i_product_name=[$21])
-                                          HiveFilter(condition=[AND(IN($17, _UTF-16LE'maroon', _UTF-16LE'burnished', _UTF-16LE'dim', _UTF-16LE'steel', _UTF-16LE'navajo', _UTF-16LE'chocolate'), BETWEEN(false, $5, 36:DECIMAL(2, 0), 45:DECIMAL(2, 0)))])
-                                            HiveTableScan(table=[[default, item]], table:alias=[item])
-                                      HiveProject(d_date_sk=[$0])
-                                        HiveFilter(condition=[=($6, 2000)])
-                                          HiveTableScan(table=[[default, date_dim]], table:alias=[d1])
-                                  HiveProject(cs_item_sk=[$0])
-                                    HiveFilter(condition=[>($1, *(2:DECIMAL(10, 0), $2))])
-                                      HiveAggregate(group=[{0}], agg#0=[sum($2)], agg#1=[sum($5)])
-                                        HiveJoin(condition=[AND(=($0, $3), =($1, $4))], joinType=[inner], algorithm=[none], cost=[not available])
-                                          HiveProject(cs_item_sk=[$15], cs_order_number=[$17], cs_ext_list_price=[$25])
-                                            HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
-                                          HiveProject(cr_item_sk=[$2], cr_order_number=[$16], +=[+(+($23, $24), $25)])
-                                            HiveTableScan(table=[[default, catalog_returns]], table:alias=[catalog_returns])
-                              HiveProject(s_store_sk=[$0], s_store_name=[$5], s_zip=[$25])
-                                HiveFilter(condition=[AND(IS NOT NULL($5), IS NOT NULL($25))])
-                                  HiveTableScan(table=[[default, store]], table:alias=[store])
-                            HiveProject(hd_demo_sk=[$0])
-                              HiveFilter(condition=[IS NOT NULL($1)])
-                                HiveTableScan(table=[[default, household_demographics]], table:alias=[hd1])
-                          HiveProject(hd_demo_sk=[$0])
-                            HiveFilter(condition=[IS NOT NULL($1)])
-                              HiveTableScan(table=[[default, household_demographics]], table:alias=[hd2])
-                        HiveProject(d_date_sk=[$0], d_year=[$6])
-                          HiveTableScan(table=[[default, date_dim]], table:alias=[d2])
-                      HiveProject(d_date_sk=[$0], d_year=[$6])
-                        HiveTableScan(table=[[default, date_dim]], table:alias=[d3])
-                    HiveProject(cd_demo_sk=[$0], cd_marital_status=[$2])
-                      HiveTableScan(table=[[default, customer_demographics]], table:alias=[cd1])
-                  HiveProject(cd_demo_sk=[$0], cd_marital_status=[$2])
-                    HiveTableScan(table=[[default, customer_demographics]], table:alias=[cd2])
-                HiveProject(ca_address_sk=[$0], ca_street_number=[$2], ca_street_name=[$3], ca_city=[$6], ca_zip=[$9])
-                  HiveTableScan(table=[[default, customer_address]], table:alias=[ad1])
-              HiveProject(ca_address_sk=[$0], ca_street_number=[$2], ca_street_name=[$3], ca_city=[$6], ca_zip=[$9])
-                HiveTableScan(table=[[default, customer_address]], table:alias=[ad2])
+    HiveProject(product_name=[$7], store_name=[$9], store_zip=[$10], b_street_number=[$11], b_streen_name=[$12], b_city=[$13], b_zip=[$14], c_street_number=[$15], c_street_name=[$16], c_city=[$17], c_zip=[$18], cnt=[$19], s1=[$20], s2=[$21], s3=[$22], s11=[$4], s21=[$5], s31=[$6], cnt1=[$3])
+      HiveJoin(condition=[AND(=($8, $0), <=($3, $19), =($9, $1), =($10, $2))], joinType=[inner], algorithm=[none], cost=[not available])
         HiveProject($f1=[$0], $f2=[$1], $f3=[$2], $f15=[$13], $f16=[$14], $f17=[$15], $f18=[$16])
-          HiveAggregate(group=[{19, 24, 25, 29, 31, 37, 38, 39, 40, 42, 43, 44, 45}], agg#0=[count()], agg#1=[sum($16)], agg#2=[sum($17)], agg#3=[sum($18)])
-            HiveJoin(condition=[=($5, $41)], joinType=[inner], algorithm=[none], cost=[not available])
-              HiveJoin(condition=[=($13, $36)], joinType=[inner], algorithm=[none], cost=[not available])
-                HiveJoin(condition=[AND(=($3, $34), <>($33, $35))], joinType=[inner], algorithm=[none], cost=[not available])
-                  HiveJoin(condition=[=($11, $32)], joinType=[inner], algorithm=[none], cost=[not available])
-                    HiveJoin(condition=[=($6, $30)], joinType=[inner], algorithm=[none], cost=[not available])
-                      HiveJoin(condition=[=($7, $28)], joinType=[inner], algorithm=[none], cost=[not available])
-                        HiveJoin(condition=[=($4, $27)], joinType=[inner], algorithm=[none], cost=[not available])
-                          HiveJoin(condition=[=($12, $26)], joinType=[inner], algorithm=[none], cost=[not available])
-                            HiveJoin(condition=[=($14, $23)], joinType=[inner], algorithm=[none], cost=[not available])
-                              HiveJoin(condition=[AND(=($9, $0), =($15, $1))], joinType=[inner], algorithm=[none], cost=[not available])
-                                HiveProject(sr_item_sk=[$2], sr_ticket_number=[$9])
-                                  HiveTableScan(table=[[default, store_returns]], table:alias=[store_returns])
-                                HiveJoin(condition=[=($7, $20)], joinType=[inner], algorithm=[none], cost=[not available])
-                                  HiveJoin(condition=[=($8, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                                    HiveProject(c_customer_sk=[$0], c_current_cdemo_sk=[$2], c_current_hdemo_sk=[$3], c_current_addr_sk=[$4], c_first_shipto_date_sk=[$5], c_first_sales_date_sk=[$6])
-                                      HiveFilter(condition=[AND(IS NOT NULL($6), IS NOT NULL($5), IS NOT NULL($2), IS NOT NULL($3), IS NOT NULL($4))])
-                                        HiveTableScan(table=[[default, customer]], table:alias=[customer])
-                                    HiveJoin(condition=[=($0, $13)], joinType=[inner], algorithm=[none], cost=[not available])
-                                      HiveJoin(condition=[=($1, $11)], joinType=[inner], algorithm=[none], cost=[not available])
-                                        HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2], ss_customer_sk=[$3], ss_cdemo_sk=[$4], ss_hdemo_sk=[$5], ss_addr_sk=[$6], ss_store_sk=[$7], ss_ticket_number=[$9], ss_wholesale_cost=[$11], ss_list_price=[$12], ss_coupon_amt=[$19])
-                                          HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7), IS NOT NULL($3), IS NOT NULL($4), IS NOT NULL($8), IS NOT NULL($5), IS NOT NULL($6))])
-                                            HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                                        HiveProject(i_item_sk=[$0], i_product_name=[$21])
-                                          HiveFilter(condition=[AND(IN($17, _UTF-16LE'maroon', _UTF-16LE'burnished', _UTF-16LE'dim', _UTF-16LE'steel', _UTF-16LE'navajo', _UTF-16LE'chocolate'), BETWEEN(false, $5, 36:DECIMAL(2, 0), 45:DECIMAL(2, 0)))])
-                                            HiveTableScan(table=[[default, item]], table:alias=[item])
-                                      HiveProject(d_date_sk=[$0])
-                                        HiveFilter(condition=[=($6, 2001)])
-                                          HiveTableScan(table=[[default, date_dim]], table:alias=[d1])
-                                  HiveProject(cs_item_sk=[$0])
-                                    HiveFilter(condition=[>($1, *(2:DECIMAL(10, 0), $2))])
-                                      HiveAggregate(group=[{0}], agg#0=[sum($2)], agg#1=[sum($5)])
-                                        HiveJoin(condition=[AND(=($0, $3), =($1, $4))], joinType=[inner], algorithm=[none], cost=[not available])
-                                          HiveProject(cs_item_sk=[$15], cs_order_number=[$17], cs_ext_list_price=[$25])
-                                            HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
-                                          HiveProject(cr_item_sk=[$2], cr_order_number=[$16], +=[+(+($23, $24), $25)])
-                                            HiveTableScan(table=[[default, catalog_returns]], table:alias=[catalog_returns])
-                              HiveProject(s_store_sk=[$0], s_store_name=[$5], s_zip=[$25])
-                                HiveFilter(condition=[AND(IS NOT NULL($5), IS NOT NULL($25))])
-                                  HiveTableScan(table=[[default, store]], table:alias=[store])
-                            HiveProject(hd_demo_sk=[$0])
-                              HiveFilter(condition=[IS NOT NULL($1)])
-                                HiveTableScan(table=[[default, household_demographics]], table:alias=[hd1])
-                          HiveProject(hd_demo_sk=[$0])
-                            HiveFilter(condition=[IS NOT NULL($1)])
-                              HiveTableScan(table=[[default, household_demographics]], table:alias=[hd2])
-                        HiveProject(d_date_sk=[$0], d_year=[$6])
-                          HiveTableScan(table=[[default, date_dim]], table:alias=[d2])
-                      HiveProject(d_date_sk=[$0], d_year=[$6])
-                        HiveTableScan(table=[[default, date_dim]], table:alias=[d3])
-                    HiveProject(cd_demo_sk=[$0], cd_marital_status=[$2])
-                      HiveTableScan(table=[[default, customer_demographics]], table:alias=[cd1])
-                  HiveProject(cd_demo_sk=[$0], cd_marital_status=[$2])
-                    HiveTableScan(table=[[default, customer_demographics]], table:alias=[cd2])
-                HiveProject(ca_address_sk=[$0], ca_street_number=[$2], ca_street_name=[$3], ca_city=[$6], ca_zip=[$9])
-                  HiveTableScan(table=[[default, customer_address]], table:alias=[ad1])
-              HiveProject(ca_address_sk=[$0], ca_street_number=[$2], ca_street_name=[$3], ca_city=[$6], ca_zip=[$9])
-                HiveTableScan(table=[[default, customer_address]], table:alias=[ad2])
+          HiveFilter(condition=[IS NOT NULL($13)])
+            HiveProject(i_item_sk=[$0], s_store_name=[$1], s_zip=[$2], ca_street_number=[$5], ca_street_name=[$6], ca_city=[$7], ca_zip=[$8], ca_street_number0=[$9], ca_street_name0=[$10], ca_city0=[$11], ca_zip0=[$12], d_year=[$3], d_year0=[$4], $f13=[$13], $f14=[$14], $f15=[$15], $f16=[$16])
+              HiveAggregate(group=[{19, 24, 25, 29, 31, 37, 38, 39, 40, 42, 43, 44, 45}], agg#0=[count()], agg#1=[sum($16)], agg#2=[sum($17)], agg#3=[sum($18)])
+                HiveJoin(condition=[=($5, $41)], joinType=[inner], algorithm=[none], cost=[not available])
+                  HiveJoin(condition=[=($13, $36)], joinType=[inner], algorithm=[none], cost=[not available])
+                    HiveJoin(condition=[AND(=($3, $34), <>($33, $35))], joinType=[inner], algorithm=[none], cost=[not available])
+                      HiveJoin(condition=[=($11, $32)], joinType=[inner], algorithm=[none], cost=[not available])
+                        HiveJoin(condition=[=($6, $30)], joinType=[inner], algorithm=[none], cost=[not available])
+                          HiveJoin(condition=[=($7, $28)], joinType=[inner], algorithm=[none], cost=[not available])
+                            HiveJoin(condition=[=($4, $27)], joinType=[inner], algorithm=[none], cost=[not available])
+                              HiveJoin(condition=[=($12, $26)], joinType=[inner], algorithm=[none], cost=[not available])
+                                HiveJoin(condition=[=($14, $23)], joinType=[inner], algorithm=[none], cost=[not available])
+                                  HiveJoin(condition=[AND(=($9, $0), =($15, $1))], joinType=[inner], algorithm=[none], cost=[not available])
+                                    HiveProject(sr_item_sk=[$2], sr_ticket_number=[$9])
+                                      HiveTableScan(table=[[default, store_returns]], table:alias=[store_returns])
+                                    HiveJoin(condition=[=($7, $20)], joinType=[inner], algorithm=[none], cost=[not available])
+                                      HiveJoin(condition=[=($8, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+                                        HiveProject(c_customer_sk=[$0], c_current_cdemo_sk=[$2], c_current_hdemo_sk=[$3], c_current_addr_sk=[$4], c_first_shipto_date_sk=[$5], c_first_sales_date_sk=[$6])
+                                          HiveFilter(condition=[AND(IS NOT NULL($6), IS NOT NULL($5), IS NOT NULL($2), IS NOT NULL($3), IS NOT NULL($4))])
+                                            HiveTableScan(table=[[default, customer]], table:alias=[customer])
+                                        HiveJoin(condition=[=($0, $13)], joinType=[inner], algorithm=[none], cost=[not available])
+                                          HiveJoin(condition=[=($1, $11)], joinType=[inner], algorithm=[none], cost=[not available])
+                                            HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2], ss_customer_sk=[$3], ss_cdemo_sk=[$4], ss_hdemo_sk=[$5], ss_addr_sk=[$6], ss_store_sk=[$7], ss_ticket_number=[$9], ss_wholesale_cost=[$11], ss_list_price=[$12], ss_coupon_amt=[$19])
+                                              HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7), IS NOT NULL($3), IS NOT NULL($4), IS NOT NULL($8), IS NOT NULL($5), IS NOT NULL($6))])
+                                                HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                                            HiveProject(i_item_sk=[$0], i_product_name=[$21])
+                                              HiveFilter(condition=[AND(IN($17, _UTF-16LE'maroon', _UTF-16LE'burnished', _UTF-16LE'dim', _UTF-16LE'steel', _UTF-16LE'navajo', _UTF-16LE'chocolate'), BETWEEN(false, $5, 36:DECIMAL(2, 0), 45:DECIMAL(2, 0)))])
+                                                HiveTableScan(table=[[default, item]], table:alias=[item])
+                                          HiveProject(d_date_sk=[$0])
+                                            HiveFilter(condition=[=($6, 2001)])
+                                              HiveTableScan(table=[[default, date_dim]], table:alias=[d1])
+                                      HiveProject(cs_item_sk=[$0])
+                                        HiveFilter(condition=[>($1, *(2:DECIMAL(10, 0), $2))])
+                                          HiveAggregate(group=[{0}], agg#0=[sum($2)], agg#1=[sum($5)])
+                                            HiveJoin(condition=[AND(=($0, $3), =($1, $4))], joinType=[inner], algorithm=[none], cost=[not available])
+                                              HiveProject(cs_item_sk=[$15], cs_order_number=[$17], cs_ext_list_price=[$25])
+                                                HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
+                                              HiveProject(cr_item_sk=[$2], cr_order_number=[$16], +=[+(+($23, $24), $25)])
+                                                HiveTableScan(table=[[default, catalog_returns]], table:alias=[catalog_returns])
+                                  HiveProject(s_store_sk=[$0], s_store_name=[$5], s_zip=[$25])
+                                    HiveFilter(condition=[AND(IS NOT NULL($5), IS NOT NULL($25))])
+                                      HiveTableScan(table=[[default, store]], table:alias=[store])
+                                HiveProject(hd_demo_sk=[$0])
+                                  HiveFilter(condition=[IS NOT NULL($1)])
+                                    HiveTableScan(table=[[default, household_demographics]], table:alias=[hd1])
+                              HiveProject(hd_demo_sk=[$0])
+                                HiveFilter(condition=[IS NOT NULL($1)])
+                                  HiveTableScan(table=[[default, household_demographics]], table:alias=[hd2])
+                            HiveProject(d_date_sk=[$0], d_year=[$6])
+                              HiveTableScan(table=[[default, date_dim]], table:alias=[d2])
+                          HiveProject(d_date_sk=[$0], d_year=[$6])
+                            HiveTableScan(table=[[default, date_dim]], table:alias=[d3])
+                        HiveProject(cd_demo_sk=[$0], cd_marital_status=[$2])
+                          HiveTableScan(table=[[default, customer_demographics]], table:alias=[cd1])
+                      HiveProject(cd_demo_sk=[$0], cd_marital_status=[$2])
+                        HiveTableScan(table=[[default, customer_demographics]], table:alias=[cd2])
+                    HiveProject(ca_address_sk=[$0], ca_street_number=[$2], ca_street_name=[$3], ca_city=[$6], ca_zip=[$9])
+                      HiveTableScan(table=[[default, customer_address]], table:alias=[ad1])
+                  HiveProject(ca_address_sk=[$0], ca_street_number=[$2], ca_street_name=[$3], ca_city=[$6], ca_zip=[$9])
+                    HiveTableScan(table=[[default, customer_address]], table:alias=[ad2])
+        HiveProject($f0=[$0], $f1=[$1], $f2=[$2], $f3=[$3], $f4=[$4], $f5=[$5], $f6=[$6], $f7=[$7], $f8=[$8], $f9=[$9], $f10=[$10], $f11=[$11], $f15=[$14], $f16=[$15], $f17=[$16], $f18=[$17])
+          HiveFilter(condition=[IS NOT NULL($14)])
+            HiveProject(i_product_name=[$1], i_item_sk=[$0], s_store_name=[$2], s_zip=[$3], ca_street_number=[$6], ca_street_name=[$7], ca_city=[$8], ca_zip=[$9], ca_street_number0=[$10], ca_street_name0=[$11], ca_city0=[$12], ca_zip0=[$13], d_year=[$4], d_year0=[$5], $f14=[$14], $f15=[$15], $f16=[$16], $f17=[$17])
+              HiveAggregate(group=[{19, 20, 24, 25, 29, 31, 37, 38, 39, 40, 42, 43, 44, 45}], agg#0=[count()], agg#1=[sum($16)], agg#2=[sum($17)], agg#3=[sum($18)])
+                HiveJoin(condition=[=($5, $41)], joinType=[inner], algorithm=[none], cost=[not available])
+                  HiveJoin(condition=[=($13, $36)], joinType=[inner], algorithm=[none], cost=[not available])
+                    HiveJoin(condition=[AND(=($3, $34), <>($33, $35))], joinType=[inner], algorithm=[none], cost=[not available])
+                      HiveJoin(condition=[=($11, $32)], joinType=[inner], algorithm=[none], cost=[not available])
+                        HiveJoin(condition=[=($6, $30)], joinType=[inner], algorithm=[none], cost=[not available])
+                          HiveJoin(condition=[=($7, $28)], joinType=[inner], algorithm=[none], cost=[not available])
+                            HiveJoin(condition=[=($4, $27)], joinType=[inner], algorithm=[none], cost=[not available])
+                              HiveJoin(condition=[=($12, $26)], joinType=[inner], algorithm=[none], cost=[not available])
+                                HiveJoin(condition=[=($14, $23)], joinType=[inner], algorithm=[none], cost=[not available])
+                                  HiveJoin(condition=[AND(=($9, $0), =($15, $1))], joinType=[inner], algorithm=[none], cost=[not available])
+                                    HiveProject(sr_item_sk=[$2], sr_ticket_number=[$9])
+                                      HiveTableScan(table=[[default, store_returns]], table:alias=[store_returns])
+                                    HiveJoin(condition=[=($7, $20)], joinType=[inner], algorithm=[none], cost=[not available])
+                                      HiveJoin(condition=[=($8, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+                                        HiveProject(c_customer_sk=[$0], c_current_cdemo_sk=[$2], c_current_hdemo_sk=[$3], c_current_addr_sk=[$4], c_first_shipto_date_sk=[$5], c_first_sales_date_sk=[$6])
+                                          HiveFilter(condition=[AND(IS NOT NULL($6), IS NOT NULL($5), IS NOT NULL($2), IS NOT NULL($3), IS NOT NULL($4))])
+                                            HiveTableScan(table=[[default, customer]], table:alias=[customer])
+                                        HiveJoin(condition=[=($0, $13)], joinType=[inner], algorithm=[none], cost=[not available])
+                                          HiveJoin(condition=[=($1, $11)], joinType=[inner], algorithm=[none], cost=[not available])
+                                            HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2], ss_customer_sk=[$3], ss_cdemo_sk=[$4], ss_hdemo_sk=[$5], ss_addr_sk=[$6], ss_store_sk=[$7], ss_ticket_number=[$9], ss_wholesale_cost=[$11], ss_list_price=[$12], ss_coupon_amt=[$19])
+                                              HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7), IS NOT NULL($3), IS NOT NULL($4), IS NOT NULL($8), IS NOT NULL($5), IS NOT NULL($6))])
+                                                HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                                            HiveProject(i_item_sk=[$0], i_product_name=[$21])
+                                              HiveFilter(condition=[AND(IN($17, _UTF-16LE'maroon', _UTF-16LE'burnished', _UTF-16LE'dim', _UTF-16LE'steel', _UTF-16LE'navajo', _UTF-16LE'chocolate'), BETWEEN(false, $5, 36:DECIMAL(2, 0), 45:DECIMAL(2, 0)))])
+                                                HiveTableScan(table=[[default, item]], table:alias=[item])
+                                          HiveProject(d_date_sk=[$0])
+                                            HiveFilter(condition=[=($6, 2000)])
+                                              HiveTableScan(table=[[default, date_dim]], table:alias=[d1])
+                                      HiveProject(cs_item_sk=[$0])
+                                        HiveFilter(condition=[>($1, *(2:DECIMAL(10, 0), $2))])
+                                          HiveAggregate(group=[{0}], agg#0=[sum($2)], agg#1=[sum($5)])
+                                            HiveJoin(condition=[AND(=($0, $3), =($1, $4))], joinType=[inner], algorithm=[none], cost=[not available])
+                                              HiveProject(cs_item_sk=[$15], cs_order_number=[$17], cs_ext_list_price=[$25])
+                                                HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
+                                              HiveProject(cr_item_sk=[$2], cr_order_number=[$16], +=[+(+($23, $24), $25)])
+                                                HiveTableScan(table=[[default, catalog_returns]], table:alias=[catalog_returns])
+                                  HiveProject(s_store_sk=[$0], s_store_name=[$5], s_zip=[$25])
+                                    HiveFilter(condition=[AND(IS NOT NULL($5), IS NOT NULL($25))])
+                                      HiveTableScan(table=[[default, store]], table:alias=[store])
+                                HiveProject(hd_demo_sk=[$0])
+                                  HiveFilter(condition=[IS NOT NULL($1)])
+                                    HiveTableScan(table=[[default, household_demographics]], table:alias=[hd1])
+                              HiveProject(hd_demo_sk=[$0])
+                                HiveFilter(condition=[IS NOT NULL($1)])
+                                  HiveTableScan(table=[[default, household_demographics]], table:alias=[hd2])
+                            HiveProject(d_date_sk=[$0], d_year=[$6])
+                              HiveTableScan(table=[[default, date_dim]], table:alias=[d2])
+                          HiveProject(d_date_sk=[$0], d_year=[$6])
+                            HiveTableScan(table=[[default, date_dim]], table:alias=[d3])
+                        HiveProject(cd_demo_sk=[$0], cd_marital_status=[$2])
+                          HiveTableScan(table=[[default, customer_demographics]], table:alias=[cd1])
+                      HiveProject(cd_demo_sk=[$0], cd_marital_status=[$2])
+                        HiveTableScan(table=[[default, customer_demographics]], table:alias=[cd2])
+                    HiveProject(ca_address_sk=[$0], ca_street_number=[$2], ca_street_name=[$3], ca_city=[$6], ca_zip=[$9])
+                      HiveTableScan(table=[[default, customer_address]], table:alias=[ad1])
+                  HiveProject(ca_address_sk=[$0], ca_street_number=[$2], ca_street_name=[$3], ca_city=[$6], ca_zip=[$9])
+                    HiveTableScan(table=[[default, customer_address]], table:alias=[ad2])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query65.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query65.q.out
index 422bbbd..4e4bfcf 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query65.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query65.q.out
@@ -66,22 +66,15 @@ POSTHOOK: Input: default@store_sales
 POSTHOOK: Output: hdfs://### HDFS PATH ###
 CBO PLAN:
 HiveSortLimit(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC], fetch=[100])
-  HiveProject(s_store_name=[$6], i_item_desc=[$8], revenue=[$2], i_current_price=[$9], i_wholesale_cost=[$10], i_brand=[$11])
-    HiveJoin(condition=[=($7, $1)], joinType=[inner], algorithm=[none], cost=[not available])
-      HiveJoin(condition=[=($5, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+  HiveProject(s_store_name=[$1], i_item_desc=[$8], revenue=[$4], i_current_price=[$9], i_wholesale_cost=[$10], i_brand=[$11])
+    HiveJoin(condition=[=($7, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+      HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+        HiveProject(s_store_sk=[$0], s_store_name=[$5])
+          HiveTableScan(table=[[default, store]], table:alias=[store])
         HiveJoin(condition=[AND(=($3, $0), <=($2, $4))], joinType=[inner], algorithm=[none], cost=[not available])
-          HiveProject(ss_store_sk=[$1], ss_item_sk=[$0], $f2=[$2])
-            HiveAggregate(group=[{1, 2}], agg#0=[sum($3)])
-              HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available])
-                HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2], ss_store_sk=[$7], ss_sales_price=[$13])
-                  HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7))])
-                    HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                HiveProject(d_date_sk=[$0])
-                  HiveFilter(condition=[BETWEEN(false, $3, 1212, 1223)])
-                    HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-          HiveProject($f0=[$0], *=[*(0.1:DECIMAL(2, 1), /($1, $2))])
-            HiveAggregate(group=[{1}], agg#0=[sum($2)], agg#1=[count($2)])
-              HiveProject(ss_item_sk=[$0], ss_store_sk=[$1], $f2=[$2])
+          HiveProject(ss_store_sk=[$0], ss_item_sk=[$1], $f2=[$2])
+            HiveFilter(condition=[IS NOT NULL($2)])
+              HiveProject(ss_store_sk=[$1], ss_item_sk=[$0], $f2=[$2])
                 HiveAggregate(group=[{1, 2}], agg#0=[sum($3)])
                   HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available])
                     HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2], ss_store_sk=[$7], ss_sales_price=[$13])
@@ -90,8 +83,18 @@ HiveSortLimit(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC], fetch=[100])
                     HiveProject(d_date_sk=[$0])
                       HiveFilter(condition=[BETWEEN(false, $3, 1212, 1223)])
                         HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-        HiveProject(s_store_sk=[$0], s_store_name=[$5])
-          HiveTableScan(table=[[default, store]], table:alias=[store])
+          HiveProject($f0=[$0], *=[*(0.1:DECIMAL(2, 1), /($1, $2))])
+            HiveFilter(condition=[AND(IS NOT NULL($1), IS NOT NULL($2))])
+              HiveAggregate(group=[{1}], agg#0=[sum($2)], agg#1=[count($2)])
+                HiveProject(ss_item_sk=[$0], ss_store_sk=[$1], $f2=[$2])
+                  HiveAggregate(group=[{1, 2}], agg#0=[sum($3)])
+                    HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available])
+                      HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2], ss_store_sk=[$7], ss_sales_price=[$13])
+                        HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($7))])
+                          HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                      HiveProject(d_date_sk=[$0])
+                        HiveFilter(condition=[BETWEEN(false, $3, 1212, 1223)])
+                          HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
       HiveProject(i_item_sk=[$0], i_item_desc=[$4], i_current_price=[$5], i_wholesale_cost=[$6], i_brand=[$8])
         HiveTableScan(table=[[default, item]], table:alias=[item])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query72.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query72.q.out
index 5c823fb..30239c6 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query72.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query72.q.out
@@ -91,12 +91,13 @@ HiveSortLimit(sort0=[$5], sort1=[$0], sort2=[$1], sort3=[$2], dir0=[DESC-nulls-l
                   HiveJoin(condition=[=($19, $2)], joinType=[inner], algorithm=[none], cost=[not available])
                     HiveJoin(condition=[AND(=($0, $14), =($8, $1), <($3, $11))], joinType=[inner], algorithm=[none], cost=[not available])
                       HiveProject(inv_date_sk=[$0], inv_item_sk=[$1], inv_warehouse_sk=[$2], inv_quantity_on_hand=[$3])
-                        HiveTableScan(table=[[default, inventory]], table:alias=[inventory])
+                        HiveFilter(condition=[IS NOT NULL($3)])
+                          HiveTableScan(table=[[default, inventory]], table:alias=[inventory])
                       HiveJoin(condition=[=($0, $12)], joinType=[inner], algorithm=[none], cost=[not available])
                         HiveJoin(condition=[=($3, $9)], joinType=[inner], algorithm=[none], cost=[not available])
                           HiveJoin(condition=[=($2, $8)], joinType=[inner], algorithm=[none], cost=[not available])
                             HiveProject(cs_sold_date_sk=[$0], cs_ship_date_sk=[$2], cs_bill_cdemo_sk=[$4], cs_bill_hdemo_sk=[$5], cs_item_sk=[$15], cs_promo_sk=[$16], cs_order_number=[$17], cs_quantity=[$18])
-                              HiveFilter(condition=[AND(IS NOT NULL($4), IS NOT NULL($5), IS NOT NULL($0), IS NOT NULL($2))])
+                              HiveFilter(condition=[AND(IS NOT NULL($4), IS NOT NULL($5), IS NOT NULL($0), IS NOT NULL($2), IS NOT NULL($18))])
                                 HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
                             HiveProject(cd_demo_sk=[$0])
                               HiveFilter(condition=[=($2, _UTF-16LE'M')])
@@ -110,14 +111,15 @@ HiveSortLimit(sort0=[$5], sort1=[$0], sort2=[$1], sort3=[$2], dir0=[DESC-nulls-l
                               HiveFilter(condition=[IS NOT NULL($4)])
                                 HiveTableScan(table=[[default, date_dim]], table:alias=[d2])
                             HiveProject(d_date_sk=[$0], d_week_seq=[$4], +=[+(CAST($2):DOUBLE, 5)])
-                              HiveFilter(condition=[AND(=($6, 2001), IS NOT NULL($4))])
+                              HiveFilter(condition=[AND(=($6, 2001), IS NOT NULL($4), IS NOT NULL(CAST($2):DOUBLE))])
                                 HiveTableScan(table=[[default, date_dim]], table:alias=[d1])
                     HiveProject(w_warehouse_sk=[$0], w_warehouse_name=[$2])
                       HiveTableScan(table=[[default, warehouse]], table:alias=[warehouse])
                   HiveProject(p_promo_sk=[$0])
                     HiveTableScan(table=[[default, promotion]], table:alias=[promotion])
                 HiveProject(d_date_sk=[$0], CAST=[CAST($2):DOUBLE])
-                  HiveTableScan(table=[[default, date_dim]], table:alias=[d3])
+                  HiveFilter(condition=[IS NOT NULL(CAST($2):DOUBLE)])
+                    HiveTableScan(table=[[default, date_dim]], table:alias=[d3])
               HiveProject(i_item_sk=[$0], i_item_desc=[$4])
                 HiveTableScan(table=[[default, item]], table:alias=[item])
           HiveProject(cr_item_sk=[$2], cr_order_number=[$16])
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query81.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query81.q.out
index c6c7a85..fd293e3 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query81.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query81.q.out
@@ -82,22 +82,9 @@ HiveProject(c_customer_id=[$0], c_salutation=[$1], c_first_name=[$2], c_last_nam
               HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
         HiveProject(cr_returning_customer_sk=[$0], ca_state=[$1], $f2=[$2], _o__c0=[$3], ctr_state=[$4])
           HiveJoin(condition=[AND(=($1, $4), >($2, $3))], joinType=[inner], algorithm=[none], cost=[not available])
-            HiveProject(cr_returning_customer_sk=[$1], ca_state=[$0], $f2=[$2])
-              HiveAggregate(group=[{1, 3}], agg#0=[sum($5)])
-                HiveJoin(condition=[=($4, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                  HiveProject(ca_address_sk=[$0], ca_state=[$8])
-                    HiveFilter(condition=[IS NOT NULL($8)])
-                      HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
-                  HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available])
-                    HiveProject(cr_returned_date_sk=[$0], cr_returning_customer_sk=[$7], cr_returning_addr_sk=[$10], cr_return_amt_inc_tax=[$20])
-                      HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($10), IS NOT NULL($7))])
-                        HiveTableScan(table=[[default, catalog_returns]], table:alias=[catalog_returns])
-                    HiveProject(d_date_sk=[$0])
-                      HiveFilter(condition=[=($6, 1998)])
-                        HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-            HiveProject(_o__c0=[*(/($1, $2), 1.2:DECIMAL(2, 1))], ctr_state=[$0])
-              HiveAggregate(group=[{0}], agg#0=[sum($2)], agg#1=[count($2)])
-                HiveProject(ca_state=[$0], cr_returning_customer_sk=[$1], $f2=[$2])
+            HiveProject(cr_returning_customer_sk=[$0], ca_state=[$1], $f2=[$2])
+              HiveFilter(condition=[IS NOT NULL($2)])
+                HiveProject(cr_returning_customer_sk=[$1], ca_state=[$0], $f2=[$2])
                   HiveAggregate(group=[{1, 3}], agg#0=[sum($5)])
                     HiveJoin(condition=[=($4, $0)], joinType=[inner], algorithm=[none], cost=[not available])
                       HiveProject(ca_address_sk=[$0], ca_state=[$8])
@@ -105,9 +92,25 @@ HiveProject(c_customer_id=[$0], c_salutation=[$1], c_first_name=[$2], c_last_nam
                           HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
                       HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available])
                         HiveProject(cr_returned_date_sk=[$0], cr_returning_customer_sk=[$7], cr_returning_addr_sk=[$10], cr_return_amt_inc_tax=[$20])
-                          HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($10))])
+                          HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($10), IS NOT NULL($7))])
                             HiveTableScan(table=[[default, catalog_returns]], table:alias=[catalog_returns])
                         HiveProject(d_date_sk=[$0])
                           HiveFilter(condition=[=($6, 1998)])
                             HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+            HiveProject(_o__c0=[*(/($1, $2), 1.2:DECIMAL(2, 1))], ctr_state=[$0])
+              HiveFilter(condition=[AND(IS NOT NULL($1), IS NOT NULL($2))])
+                HiveAggregate(group=[{0}], agg#0=[sum($2)], agg#1=[count($2)])
+                  HiveProject(ca_state=[$0], cr_returning_customer_sk=[$1], $f2=[$2])
+                    HiveAggregate(group=[{1, 3}], agg#0=[sum($5)])
+                      HiveJoin(condition=[=($4, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+                        HiveProject(ca_address_sk=[$0], ca_state=[$8])
+                          HiveFilter(condition=[IS NOT NULL($8)])
+                            HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
+                        HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available])
+                          HiveProject(cr_returned_date_sk=[$0], cr_returning_customer_sk=[$7], cr_returning_addr_sk=[$10], cr_return_amt_inc_tax=[$20])
+                            HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($10))])
+                              HiveTableScan(table=[[default, catalog_returns]], table:alias=[catalog_returns])
+                          HiveProject(d_date_sk=[$0])
+                            HiveFilter(condition=[=($6, 1998)])
+                              HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query92.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query92.q.out
index 93208d0..80a1018 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query92.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query92.q.out
@@ -66,26 +66,26 @@ POSTHOOK: Input: default@web_sales
 POSTHOOK: Output: hdfs://### HDFS PATH ###
 CBO PLAN:
 HiveAggregate(group=[{}], agg#0=[sum($2)])
-  HiveJoin(condition=[AND(>($2, $5), =($6, $1))], joinType=[inner], algorithm=[none], cost=[not available])
-    HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-      HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$3], ws_ext_discount_amt=[$22])
-        HiveFilter(condition=[IS NOT NULL($0)])
-          HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
-      HiveProject(d_date_sk=[$0])
-        HiveFilter(condition=[BETWEEN(false, CAST($2):TIMESTAMP(9), 1998-03-18 00:00:00:TIMESTAMP(9), 1998-06-16 00:00:00:TIMESTAMP(9))])
-          HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-    HiveProject(ws_item_sk=[$0], CAST3=[$1], i_item_sk=[$2])
-      HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-        HiveProject(ws_item_sk=[$0], CAST3=[CAST(*(1.3:DECIMAL(2, 1), /($1, $2))):DECIMAL(14, 7)])
-          HiveAggregate(group=[{1}], agg#0=[sum($2)], agg#1=[count($2)])
-            HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-              HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$3], ws_ext_discount_amt=[$22])
-                HiveFilter(condition=[IS NOT NULL($0)])
-                  HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
-              HiveProject(d_date_sk=[$0])
-                HiveFilter(condition=[BETWEEN(false, CAST($2):TIMESTAMP(9), 1998-03-18 00:00:00:TIMESTAMP(9), 1998-06-16 00:00:00:TIMESTAMP(9))])
-                  HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-        HiveProject(i_item_sk=[$0])
-          HiveFilter(condition=[=($13, 269)])
-            HiveTableScan(table=[[default, item]], table:alias=[item])
+  HiveJoin(condition=[AND(=($5, $4), >($2, $6))], joinType=[inner], algorithm=[none], cost=[not available])
+    HiveJoin(condition=[=($4, $1)], joinType=[inner], algorithm=[none], cost=[not available])
+      HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+        HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$3], ws_ext_discount_amt=[$22])
+          HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($22))])
+            HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
+        HiveProject(d_date_sk=[$0])
+          HiveFilter(condition=[BETWEEN(false, CAST($2):TIMESTAMP(9), 1998-03-18 00:00:00:TIMESTAMP(9), 1998-06-16 00:00:00:TIMESTAMP(9))])
+            HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+      HiveProject(i_item_sk=[$0])
+        HiveFilter(condition=[=($13, 269)])
+          HiveTableScan(table=[[default, item]], table:alias=[item])
+    HiveProject(ws_item_sk=[$0], CAST3=[CAST(*(1.3:DECIMAL(2, 1), /($1, $2))):DECIMAL(14, 7)])
+      HiveFilter(condition=[IS NOT NULL(CAST(*(1.3:DECIMAL(2, 1), /($1, $2))):DECIMAL(14, 7))])
+        HiveAggregate(group=[{1}], agg#0=[sum($2)], agg#1=[count($2)])
+          HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+            HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$3], ws_ext_discount_amt=[$22])
+              HiveFilter(condition=[IS NOT NULL($0)])
+                HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
+            HiveProject(d_date_sk=[$0])
+              HiveFilter(condition=[BETWEEN(false, CAST($2):TIMESTAMP(9), 1998-03-18 00:00:00:TIMESTAMP(9), 1998-06-16 00:00:00:TIMESTAMP(9))])
+                HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/mv_query44.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/mv_query44.q.out
index c7bfb39..66915cc 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/mv_query44.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/mv_query44.q.out
@@ -19,7 +19,8 @@ POSTHOOK: type: CREATE_MATERIALIZED_VIEW
 POSTHOOK: Input: default@store_sales
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@mv_store_sales_item_customer
-Warning: Shuffle Join MERGEJOIN[103][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[153][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product
+Warning: Shuffle Join MERGEJOIN[154][tables = [$hdt$_2, $hdt$_3, $hdt$_1]] in Stage 'Reducer 5' is a cross product
 PREHOOK: query: explain
 select  asceding.rnk, i1.i_product_name best_performing, i2.i_product_name worst_performing
 from(select *
@@ -97,118 +98,155 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 Plan optimized by CBO.
 
 Vertex dependency in root stage
-Reducer 11 <- Map 10 (SIMPLE_EDGE)
+Reducer 10 <- Reducer 9 (SIMPLE_EDGE)
+Reducer 11 <- Reducer 5 (SIMPLE_EDGE)
+Reducer 13 <- Map 12 (SIMPLE_EDGE)
+Reducer 15 <- Map 14 (SIMPLE_EDGE)
 Reducer 2 <- Map 1 (SIMPLE_EDGE)
-Reducer 3 <- Reducer 11 (CUSTOM_SIMPLE_EDGE), Reducer 2 (CUSTOM_SIMPLE_EDGE)
-Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
-Reducer 5 <- Reducer 4 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE)
-Reducer 6 <- Map 12 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
-Reducer 7 <- Map 12 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
-Reducer 8 <- Reducer 7 (SIMPLE_EDGE)
-Reducer 9 <- Reducer 3 (SIMPLE_EDGE)
+Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE)
+Reducer 4 <- Reducer 13 (CUSTOM_SIMPLE_EDGE), Reducer 3 (CUSTOM_SIMPLE_EDGE)
+Reducer 5 <- Reducer 15 (CUSTOM_SIMPLE_EDGE), Reducer 4 (CUSTOM_SIMPLE_EDGE)
+Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
+Reducer 7 <- Reducer 11 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
+Reducer 8 <- Map 16 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE)
+Reducer 9 <- Map 16 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
     limit:100
     Stage-1
-      Reducer 8 vectorized
-      File Output Operator [FS_137]
-        Limit [LIM_136] (rows=100 width=218)
+      Reducer 10 vectorized
+      File Output Operator [FS_204]
+        Limit [LIM_203] (rows=100 width=218)
           Number of rows:100
-          Select Operator [SEL_135] (rows=6951 width=218)
+          Select Operator [SEL_202] (rows=6951 width=218)
             Output:["_col0","_col1","_col2"]
-          <-Reducer 7 [SIMPLE_EDGE]
-            SHUFFLE [RS_66]
-              Select Operator [SEL_65] (rows=6951 width=218)
+          <-Reducer 9 [SIMPLE_EDGE]
+            SHUFFLE [RS_106]
+              Select Operator [SEL_105] (rows=6951 width=218)
                 Output:["_col0","_col1","_col2"]
-                Merge Join Operator [MERGEJOIN_107] (rows=6951 width=218)
-                  Conds:RS_62._col2=RS_134._col0(Inner),Output:["_col1","_col5","_col7"]
-                <-Map 12 [SIMPLE_EDGE] vectorized
-                  SHUFFLE [RS_134]
+                Merge Join Operator [MERGEJOIN_159] (rows=6951 width=218)
+                  Conds:RS_102._col2=RS_201._col0(Inner),Output:["_col1","_col5","_col7"]
+                <-Map 16 [SIMPLE_EDGE] vectorized
+                  SHUFFLE [RS_201]
                     PartitionCols:_col0
-                    Select Operator [SEL_132] (rows=462000 width=111)
+                    Select Operator [SEL_199] (rows=462000 width=111)
                       Output:["_col0","_col1"]
-                      TableScan [TS_52] (rows=462000 width=111)
+                      TableScan [TS_92] (rows=462000 width=111)
                         default@item,i1,Tbl:COMPLETE,Col:COMPLETE,Output:["i_item_sk","i_product_name"]
-                <-Reducer 6 [SIMPLE_EDGE]
-                  SHUFFLE [RS_62]
+                <-Reducer 8 [SIMPLE_EDGE]
+                  SHUFFLE [RS_102]
                     PartitionCols:_col2
-                    Merge Join Operator [MERGEJOIN_106] (rows=6951 width=115)
-                      Conds:RS_59._col0=RS_133._col0(Inner),Output:["_col1","_col2","_col5"]
-                    <-Map 12 [SIMPLE_EDGE] vectorized
-                      SHUFFLE [RS_133]
+                    Merge Join Operator [MERGEJOIN_158] (rows=6951 width=115)
+                      Conds:RS_99._col0=RS_200._col0(Inner),Output:["_col1","_col2","_col5"]
+                    <-Map 16 [SIMPLE_EDGE] vectorized
+                      SHUFFLE [RS_200]
                         PartitionCols:_col0
-                         Please refer to the previous Select Operator [SEL_132]
-                    <-Reducer 5 [SIMPLE_EDGE]
-                      SHUFFLE [RS_59]
+                         Please refer to the previous Select Operator [SEL_199]
+                    <-Reducer 7 [SIMPLE_EDGE]
+                      SHUFFLE [RS_99]
                         PartitionCols:_col0
-                        Merge Join Operator [MERGEJOIN_105] (rows=6951 width=12)
-                          Conds:RS_126._col1=RS_131._col1(Inner),Output:["_col0","_col1","_col2"]
-                        <-Reducer 4 [SIMPLE_EDGE] vectorized
-                          SHUFFLE [RS_126]
+                        Merge Join Operator [MERGEJOIN_157] (rows=6951 width=12)
+                          Conds:RS_193._col1=RS_198._col1(Inner),Output:["_col0","_col1","_col2"]
+                        <-Reducer 11 [SIMPLE_EDGE] vectorized
+                          SHUFFLE [RS_198]
                             PartitionCols:_col1
-                            Select Operator [SEL_125] (rows=6951 width=8)
+                            Select Operator [SEL_197] (rows=6951 width=8)
                               Output:["_col0","_col1"]
-                              Filter Operator [FIL_124] (rows=6951 width=116)
+                              Filter Operator [FIL_196] (rows=6951 width=116)
                                 predicate:(rank_window_0 < 11)
-                                PTF Operator [PTF_123] (rows=20854 width=116)
-                                  Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col1 ASC NULLS FIRST","partition by:":"0"}]
-                                  Select Operator [SEL_122] (rows=20854 width=116)
-                                    Output:["_col0","_col1"]
-                                  <-Reducer 3 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_19]
+                                PTF Operator [PTF_195] (rows=20854 width=116)
+                                  Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col3 DESC NULLS LAST","partition by:":"0"}]
+                                  Select Operator [SEL_194] (rows=20854 width=116)
+                                    Output:["_col2","_col3"]
+                                  <-Reducer 5 [SIMPLE_EDGE]
+                                    SHUFFLE [RS_85]
                                       PartitionCols:0
-                                      Filter Operator [FIL_18] (rows=20854 width=228)
-                                        predicate:(_col1 > _col2)
-                                        Merge Join Operator [MERGEJOIN_103] (rows=62562 width=228)
-                                          Conds:(Inner),Output:["_col0","_col1","_col2"]
-                                        <-Reducer 11 [CUSTOM_SIMPLE_EDGE] vectorized
-                                          PARTITION_ONLY_SHUFFLE [RS_121]
-                                            Select Operator [SEL_120] (rows=1 width=112)
-                                              Output:["_col0"]
-                                              Group By Operator [GBY_119] (rows=1 width=124)
-                                                Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)"],keys:KEY._col0
-                                              <-Map 10 [SIMPLE_EDGE] vectorized
-                                                SHUFFLE [RS_118]
-                                                  PartitionCols:_col0
-                                                  Group By Operator [GBY_117] (rows=258 width=124)
-                                                    Output:["_col0","_col1","_col2"],aggregations:["sum(_col1)","count(_col1)"],keys:true
-                                                    Select Operator [SEL_116] (rows=287946 width=114)
-                                                      Output:["_col1"]
-                                                      Filter Operator [FIL_115] (rows=287946 width=114)
-                                                        predicate:((ss_store_sk = 410) and ss_hdemo_sk is null)
-                                                        TableScan [TS_7] (rows=575995635 width=114)
-                                                          default@store_sales,store_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ss_hdemo_sk","ss_store_sk","ss_net_profit"]
-                                        <-Reducer 2 [CUSTOM_SIMPLE_EDGE] vectorized
-                                          PARTITION_ONLY_SHUFFLE [RS_114]
-                                            Select Operator [SEL_113] (rows=62562 width=116)
+                                      Filter Operator [FIL_38] (rows=20854 width=228)
+                                        predicate:(_col3 > _col1)
+                                        Merge Join Operator [MERGEJOIN_154] (rows=62562 width=228)
+                                          Conds:(Inner),Output:["_col1","_col2","_col3"]
+                                        <-Reducer 15 [CUSTOM_SIMPLE_EDGE] vectorized
+                                          PARTITION_ONLY_SHUFFLE [RS_188]
+                                            Select Operator [SEL_187] (rows=62562 width=116)
                                               Output:["_col0","_col1"]
-                                              Group By Operator [GBY_112] (rows=62562 width=124)
-                                                Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)"],keys:KEY._col0
-                                              <-Map 1 [SIMPLE_EDGE] vectorized
-                                                SHUFFLE [RS_111]
-                                                  PartitionCols:_col0
-                                                  Group By Operator [GBY_110] (rows=3199976 width=124)
-                                                    Output:["_col0","_col1","_col2"],aggregations:["sum(ss_net_profit)","count(ss_net_profit)"],keys:ss_item_sk
-                                                    Select Operator [SEL_109] (rows=6399952 width=114)
-                                                      Output:["ss_item_sk","ss_net_profit"]
-                                                      Filter Operator [FIL_108] (rows=6399952 width=114)
-                                                        predicate:(ss_store_sk = 410)
-                                                        TableScan [TS_0] (rows=575995635 width=114)
-                                                          default@store_sales,ss1,Tbl:COMPLETE,Col:COMPLETE,Output:["ss_item_sk","ss_store_sk","ss_net_profit"]
-                        <-Reducer 9 [SIMPLE_EDGE] vectorized
-                          SHUFFLE [RS_131]
+                                              Filter Operator [FIL_186] (rows=62562 width=124)
+                                                predicate:(_col1 is not null and _col2 is not null)
+                                                Group By Operator [GBY_185] (rows=62562 width=124)
+                                                  Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)"],keys:KEY._col0
+                                                <-Map 14 [SIMPLE_EDGE] vectorized
+                                                  SHUFFLE [RS_184]
+                                                    PartitionCols:_col0
+                                                    Group By Operator [GBY_183] (rows=3199976 width=124)
+                                                      Output:["_col0","_col1","_col2"],aggregations:["sum(ss_net_profit)","count(ss_net_profit)"],keys:ss_item_sk
+                                                      Select Operator [SEL_182] (rows=6399952 width=114)
+                                                        Output:["ss_item_sk","ss_net_profit"]
+                                                        Filter Operator [FIL_181] (rows=6399952 width=114)
+                                                          predicate:(ss_store_sk = 410)
+                                                          TableScan [TS_24] (rows=575995635 width=114)
+                                                            default@store_sales,ss1,Tbl:COMPLETE,Col:COMPLETE,Output:["ss_item_sk","ss_store_sk","ss_net_profit"]
+                                        <-Reducer 4 [CUSTOM_SIMPLE_EDGE]
+                                          PARTITION_ONLY_SHUFFLE [RS_35]
+                                            Merge Join Operator [MERGEJOIN_153] (rows=1 width=112)
+                                              Conds:(Inner),Output:["_col1"]
+                                            <-Reducer 13 [CUSTOM_SIMPLE_EDGE] vectorized
+                                              PARTITION_ONLY_SHUFFLE [RS_180]
+                                                Select Operator [SEL_179] (rows=1 width=112)
+                                                  Output:["_col0"]
+                                                  Filter Operator [FIL_178] (rows=1 width=120)
+                                                    predicate:(_col1 is not null and _col2 is not null)
+                                                    Select Operator [SEL_177] (rows=1 width=120)
+                                                      Output:["_col1","_col2"]
+                                                      Group By Operator [GBY_176] (rows=1 width=124)
+                                                        Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)"],keys:KEY._col0
+                                                      <-Map 12 [SIMPLE_EDGE] vectorized
+                                                        SHUFFLE [RS_175]
+                                                          PartitionCols:_col0
+                                                          Group By Operator [GBY_174] (rows=258 width=124)
+                                                            Output:["_col0","_col1","_col2"],aggregations:["sum(_col1)","count(_col1)"],keys:true
+                                                            Select Operator [SEL_173] (rows=287946 width=114)
+                                                              Output:["_col1"]
+                                                              Filter Operator [FIL_172] (rows=287946 width=114)
+                                                                predicate:((ss_store_sk = 410) and ss_hdemo_sk is null)
+                                                                TableScan [TS_15] (rows=575995635 width=114)
+                                                                  default@store_sales,store_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ss_hdemo_sk","ss_store_sk","ss_net_profit"]
+                                            <-Reducer 3 [CUSTOM_SIMPLE_EDGE] vectorized
+                                              PARTITION_ONLY_SHUFFLE [RS_171]
+                                                Select Operator [SEL_170] (rows=1 width=8)
+                                                  Filter Operator [FIL_169] (rows=1 width=8)
+                                                    predicate:(sq_count_check(_col0) <= 1)
+                                                    Group By Operator [GBY_168] (rows=1 width=8)
+                                                      Output:["_col0"],aggregations:["count(VALUE._col0)"]
+                                                    <-Reducer 2 [CUSTOM_SIMPLE_EDGE] vectorized
+                                                      PARTITION_ONLY_SHUFFLE [RS_167]
+                                                        Group By Operator [GBY_166] (rows=1 width=8)
+                                                          Output:["_col0"],aggregations:["count()"]
+                                                          Select Operator [SEL_165] (rows=1 width=4)
+                                                            Group By Operator [GBY_164] (rows=1 width=4)
+                                                              Output:["_col0"],keys:KEY._col0
+                                                            <-Map 1 [SIMPLE_EDGE] vectorized
+                                                              SHUFFLE [RS_163]
+                                                                PartitionCols:_col0
+                                                                Group By Operator [GBY_162] (rows=18 width=4)
+                                                                  Output:["_col0"],keys:true
+                                                                  Select Operator [SEL_161] (rows=287946 width=7)
+                                                                    Filter Operator [FIL_160] (rows=287946 width=7)
+                                                                      predicate:((ss_store_sk = 410) and ss_hdemo_sk is null)
+                                                                      TableScan [TS_0] (rows=575995635 width=7)
+                                                                        default@store_sales,store_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ss_hdemo_sk","ss_store_sk"]
+                        <-Reducer 6 [SIMPLE_EDGE] vectorized
+                          SHUFFLE [RS_193]
                             PartitionCols:_col1
-                            Select Operator [SEL_130] (rows=6951 width=8)
+                            Select Operator [SEL_192] (rows=6951 width=8)
                               Output:["_col0","_col1"]
-                              Filter Operator [FIL_129] (rows=6951 width=116)
+                              Filter Operator [FIL_191] (rows=6951 width=116)
                                 predicate:(rank_window_0 < 11)
-                                PTF Operator [PTF_128] (rows=20854 width=116)
-                                  Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col1 DESC NULLS LAST","partition by:":"0"}]
-                                  Select Operator [SEL_127] (rows=20854 width=116)
-                                    Output:["_col0","_col1"]
-                                  <-Reducer 3 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_45]
+                                PTF Operator [PTF_190] (rows=20854 width=116)
+                                  Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col3 ASC NULLS FIRST","partition by:":"0"}]
+                                  Select Operator [SEL_189] (rows=20854 width=116)
+                                    Output:["_col2","_col3"]
+                                  <-Reducer 5 [SIMPLE_EDGE]
+                                    SHUFFLE [RS_39]
                                       PartitionCols:0
-                                       Please refer to the previous Filter Operator [FIL_18]
+                                       Please refer to the previous Filter Operator [FIL_38]
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query1.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query1.q.out
index d1fd7a5..eac858d 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query1.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query1.q.out
@@ -59,119 +59,123 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 Plan optimized by CBO.
 
 Vertex dependency in root stage
-Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 10 (SIMPLE_EDGE)
-Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
-Reducer 4 <- Map 11 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
-Reducer 5 <- Reducer 4 (ONE_TO_ONE_EDGE), Reducer 9 (SIMPLE_EDGE)
-Reducer 6 <- Map 12 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
-Reducer 7 <- Reducer 6 (SIMPLE_EDGE)
-Reducer 8 <- Map 1 (SIMPLE_EDGE), Map 10 (SIMPLE_EDGE)
-Reducer 9 <- Reducer 8 (SIMPLE_EDGE)
+Reducer 10 <- Reducer 9 (SIMPLE_EDGE)
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE)
+Reducer 3 <- Reducer 10 (SIMPLE_EDGE), Reducer 2 (ONE_TO_ONE_EDGE)
+Reducer 4 <- Map 12 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+Reducer 5 <- Reducer 4 (SIMPLE_EDGE)
+Reducer 7 <- Map 11 (SIMPLE_EDGE), Map 6 (SIMPLE_EDGE)
+Reducer 8 <- Reducer 7 (SIMPLE_EDGE)
+Reducer 9 <- Map 11 (SIMPLE_EDGE), Map 6 (SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
     limit:100
     Stage-1
-      Reducer 7 vectorized
-      File Output Operator [FS_160]
-        Limit [LIM_159] (rows=100 width=100)
+      Reducer 5 vectorized
+      File Output Operator [FS_164]
+        Limit [LIM_163] (rows=100 width=100)
           Number of rows:100
-          Select Operator [SEL_158] (rows=816091 width=100)
+          Select Operator [SEL_162] (rows=816091 width=100)
             Output:["_col0"]
-          <-Reducer 6 [SIMPLE_EDGE]
-            SHUFFLE [RS_49]
-              Select Operator [SEL_48] (rows=816091 width=100)
+          <-Reducer 4 [SIMPLE_EDGE]
+            SHUFFLE [RS_51]
+              Select Operator [SEL_50] (rows=816091 width=100)
                 Output:["_col0"]
-                Merge Join Operator [MERGEJOIN_134] (rows=816091 width=100)
-                  Conds:RS_45._col0=RS_157._col0(Inner),Output:["_col7"]
+                Merge Join Operator [MERGEJOIN_136] (rows=816091 width=100)
+                  Conds:RS_47._col1=RS_161._col0(Inner),Output:["_col7"]
                 <-Map 12 [SIMPLE_EDGE] vectorized
-                  SHUFFLE [RS_157]
+                  SHUFFLE [RS_161]
                     PartitionCols:_col0
-                    Select Operator [SEL_156] (rows=80000000 width=104)
+                    Select Operator [SEL_160] (rows=80000000 width=104)
                       Output:["_col0","_col1"]
-                      TableScan [TS_36] (rows=80000000 width=104)
+                      TableScan [TS_38] (rows=80000000 width=104)
                         default@customer,customer,Tbl:COMPLETE,Col:COMPLETE,Output:["c_customer_sk","c_customer_id"]
-                <-Reducer 5 [SIMPLE_EDGE]
-                  SHUFFLE [RS_45]
-                    PartitionCols:_col0
-                    Filter Operator [FIL_44] (rows=816091 width=225)
-                      predicate:(_col2 > _col4)
-                      Merge Join Operator [MERGEJOIN_133] (rows=2448274 width=225)
-                        Conds:RS_41._col1=RS_155._col1(Inner),Output:["_col0","_col2","_col4"]
-                      <-Reducer 4 [ONE_TO_ONE_EDGE]
-                        FORWARD [RS_41]
+                <-Reducer 3 [SIMPLE_EDGE]
+                  SHUFFLE [RS_47]
+                    PartitionCols:_col1
+                    Filter Operator [FIL_46] (rows=816091 width=225)
+                      predicate:(_col3 > _col4)
+                      Merge Join Operator [MERGEJOIN_135] (rows=2448274 width=225)
+                        Conds:RS_43._col2=RS_159._col1(Inner),Output:["_col1","_col3","_col4"]
+                      <-Reducer 10 [SIMPLE_EDGE] vectorized
+                        SHUFFLE [RS_159]
                           PartitionCols:_col1
-                          Merge Join Operator [MERGEJOIN_131] (rows=2369298 width=114)
-                            Conds:RS_147._col1=RS_150._col0(Inner),Output:["_col0","_col1","_col2"]
-                          <-Map 11 [SIMPLE_EDGE] vectorized
-                            SHUFFLE [RS_150]
+                          Select Operator [SEL_158] (rows=31 width=115)
+                            Output:["_col0","_col1"]
+                            Filter Operator [FIL_157] (rows=31 width=123)
+                              predicate:(_col1 is not null and _col2 is not null)
+                              Group By Operator [GBY_156] (rows=31 width=123)
+                                Output:["_col0","_col1","_col2"],aggregations:["sum(_col2)","count(_col2)"],keys:_col1
+                                Select Operator [SEL_155] (rows=14291868 width=119)
+                                  Output:["_col1","_col2"]
+                                  Group By Operator [GBY_154] (rows=14291868 width=119)
+                                    Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, KEY._col1
+                                  <-Reducer 9 [SIMPLE_EDGE]
+                                    SHUFFLE [RS_29]
+                                      PartitionCols:_col0
+                                      Group By Operator [GBY_28] (rows=17467258 width=119)
+                                        Output:["_col0","_col1","_col2"],aggregations:["sum(_col3)"],keys:_col2, _col1
+                                        Merge Join Operator [MERGEJOIN_134] (rows=17467258 width=107)
+                                          Conds:RS_145._col0=RS_149._col0(Inner),Output:["_col1","_col2","_col3"]
+                                        <-Map 11 [SIMPLE_EDGE] vectorized
+                                          SHUFFLE [RS_149]
+                                            PartitionCols:_col0
+                                            Select Operator [SEL_147] (rows=652 width=4)
+                                              Output:["_col0"]
+                                              Filter Operator [FIL_146] (rows=652 width=8)
+                                                predicate:(d_year = 2000)
+                                                TableScan [TS_6] (rows=73049 width=8)
+                                                  default@date_dim,date_dim,Tbl:COMPLETE,Col:COMPLETE,Output:["d_date_sk","d_year"]
+                                        <-Map 6 [SIMPLE_EDGE] vectorized
+                                          SHUFFLE [RS_145]
+                                            PartitionCols:_col0
+                                            Select Operator [SEL_143] (rows=53634860 width=119)
+                                              Output:["_col0","_col1","_col2","_col3"]
+                                              Filter Operator [FIL_141] (rows=53634860 width=119)
+                                                predicate:(sr_returned_date_sk is not null and sr_store_sk is not null)
+                                                TableScan [TS_3] (rows=57591150 width=119)
+                                                  default@store_returns,store_returns,Tbl:COMPLETE,Col:COMPLETE,Output:["sr_returned_date_sk","sr_customer_sk","sr_store_sk","sr_fee"]
+                      <-Reducer 2 [ONE_TO_ONE_EDGE]
+                        FORWARD [RS_43]
+                          PartitionCols:_col2
+                          Merge Join Operator [MERGEJOIN_133] (rows=2369298 width=114)
+                            Conds:RS_139._col0=RS_153._col1(Inner),Output:["_col1","_col2","_col3"]
+                          <-Map 1 [SIMPLE_EDGE] vectorized
+                            SHUFFLE [RS_139]
                               PartitionCols:_col0
-                              Select Operator [SEL_149] (rows=35 width=4)
+                              Select Operator [SEL_138] (rows=35 width=4)
                                 Output:["_col0"]
-                                Filter Operator [FIL_148] (rows=35 width=90)
+                                Filter Operator [FIL_137] (rows=35 width=90)
                                   predicate:(s_state = 'NM')
-                                  TableScan [TS_14] (rows=1704 width=90)
+                                  TableScan [TS_0] (rows=1704 width=90)
                                     default@store,store,Tbl:COMPLETE,Col:COMPLETE,Output:["s_store_sk","s_state"]
-                          <-Reducer 3 [SIMPLE_EDGE] vectorized
-                            SHUFFLE [RS_147]
+                          <-Reducer 8 [SIMPLE_EDGE] vectorized
+                            SHUFFLE [RS_153]
                               PartitionCols:_col1
-                              Select Operator [SEL_146] (rows=14291868 width=119)
-                                Output:["_col0","_col1","_col2"]
-                                Group By Operator [GBY_145] (rows=14291868 width=119)
-                                  Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, KEY._col1
-                                <-Reducer 2 [SIMPLE_EDGE]
-                                  SHUFFLE [RS_11]
-                                    PartitionCols:_col0, _col1
-                                    Group By Operator [GBY_10] (rows=16855704 width=119)
-                                      Output:["_col0","_col1","_col2"],aggregations:["sum(_col3)"],keys:_col2, _col1
-                                      Merge Join Operator [MERGEJOIN_130] (rows=16855704 width=107)
-                                        Conds:RS_139._col0=RS_143._col0(Inner),Output:["_col1","_col2","_col3"]
-                                      <-Map 1 [SIMPLE_EDGE] vectorized
-                                        SHUFFLE [RS_139]
-                                          PartitionCols:_col0
-                                          Select Operator [SEL_137] (rows=51757026 width=119)
-                                            Output:["_col0","_col1","_col2","_col3"]
-                                            Filter Operator [FIL_135] (rows=51757026 width=119)
-                                              predicate:(sr_customer_sk is not null and sr_returned_date_sk is not null and sr_store_sk is not null)
-                                              TableScan [TS_0] (rows=57591150 width=119)
-                                                default@store_returns,store_returns,Tbl:COMPLETE,Col:COMPLETE,Output:["sr_returned_date_sk","sr_customer_sk","sr_store_sk","sr_fee"]
-                                      <-Map 10 [SIMPLE_EDGE] vectorized
-                                        SHUFFLE [RS_143]
-                                          PartitionCols:_col0
-                                          Select Operator [SEL_142] (rows=652 width=4)
-                                            Output:["_col0"]
-                                            Filter Operator [FIL_141] (rows=652 width=8)
-                                              predicate:(d_year = 2000)
-                                              TableScan [TS_3] (rows=73049 width=8)
-                                                default@date_dim,date_dim,Tbl:COMPLETE,Col:COMPLETE,Output:["d_date_sk","d_year"]
-                      <-Reducer 9 [SIMPLE_EDGE] vectorized
-                        SHUFFLE [RS_155]
-                          PartitionCols:_col1
-                          Select Operator [SEL_154] (rows=31 width=115)
-                            Output:["_col0","_col1"]
-                            Group By Operator [GBY_153] (rows=31 width=123)
-                              Output:["_col0","_col1","_col2"],aggregations:["sum(_col2)","count(_col2)"],keys:_col1
                               Select Operator [SEL_152] (rows=14291868 width=119)
-                                Output:["_col1","_col2"]
-                                Group By Operator [GBY_151] (rows=14291868 width=119)
-                                  Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, KEY._col1
-                                <-Reducer 8 [SIMPLE_EDGE]
-                                  SHUFFLE [RS_28]
-                                    PartitionCols:_col0
-                                    Group By Operator [GBY_27] (rows=17467258 width=119)
-                                      Output:["_col0","_col1","_col2"],aggregations:["sum(_col3)"],keys:_col2, _col1
-                                      Merge Join Operator [MERGEJOIN_132] (rows=17467258 width=107)
-                                        Conds:RS_140._col0=RS_144._col0(Inner),Output:["_col1","_col2","_col3"]
-                                      <-Map 1 [SIMPLE_EDGE] vectorized
-                                        SHUFFLE [RS_140]
-                                          PartitionCols:_col0
-                                          Select Operator [SEL_138] (rows=53634860 width=119)
-                                            Output:["_col0","_col1","_col2","_col3"]
-                                            Filter Operator [FIL_136] (rows=53634860 width=119)
-                                              predicate:(sr_returned_date_sk is not null and sr_store_sk is not null)
-                                               Please refer to the previous TableScan [TS_0]
-                                      <-Map 10 [SIMPLE_EDGE] vectorized
-                                        SHUFFLE [RS_144]
-                                          PartitionCols:_col0
-                                           Please refer to the previous Select Operator [SEL_142]
+                                Output:["_col0","_col1","_col2"]
+                                Filter Operator [FIL_151] (rows=14291868 width=119)
+                                  predicate:_col2 is not null
+                                  Group By Operator [GBY_150] (rows=14291868 width=119)
+                                    Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, KEY._col1
+                                  <-Reducer 7 [SIMPLE_EDGE]
+                                    SHUFFLE [RS_14]
+                                      PartitionCols:_col0, _col1
+                                      Group By Operator [GBY_13] (rows=16855704 width=119)
+                                        Output:["_col0","_col1","_col2"],aggregations:["sum(_col3)"],keys:_col2, _col1
+                                        Merge Join Operator [MERGEJOIN_132] (rows=16855704 width=107)
+                                          Conds:RS_144._col0=RS_148._col0(Inner),Output:["_col1","_col2","_col3"]
+                                        <-Map 11 [SIMPLE_EDGE] vectorized
+                                          SHUFFLE [RS_148]
+                                            PartitionCols:_col0
+                                             Please refer to the previous Select Operator [SEL_147]
+                                        <-Map 6 [SIMPLE_EDGE] vectorized
+                                          SHUFFLE [RS_144]
+                                            PartitionCols:_col0
+                                            Select Operator [SEL_142] (rows=51757026 width=119)
+                                              Output:["_col0","_col1","_col2","_col3"]
+                                              Filter Operator [FIL_140] (rows=51757026 width=119)
+                                                predicate:(sr_customer_sk is not null and sr_returned_date_sk is not null and sr_store_sk is not null)
+                                                 Please refer to the previous TableScan [TS_3]
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query14.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query14.q.out
index 0f48872..09d2ecb 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query14.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query14.q.out
@@ -1,6 +1,6 @@
-Warning: Shuffle Join MERGEJOIN[1182][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 6' is a cross product
-Warning: Shuffle Join MERGEJOIN[1189][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 16' is a cross product
-Warning: Shuffle Join MERGEJOIN[1196][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 22' is a cross product
+Warning: Shuffle Join MERGEJOIN[1191][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 6' is a cross product
+Warning: Shuffle Join MERGEJOIN[1198][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 16' is a cross product
+Warning: Shuffle Join MERGEJOIN[1205][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 22' is a cross product
 PREHOOK: query: explain
 with  cross_items as
  (select i_item_sk ss_item_sk
@@ -296,754 +296,766 @@ Stage-0
     limit:100
     Stage-1
       Reducer 9 vectorized
-      File Output Operator [FS_1350]
-        Limit [LIM_1349] (rows=7 width=212)
+      File Output Operator [FS_1361]
+        Limit [LIM_1360] (rows=7 width=212)
           Number of rows:100
-          Select Operator [SEL_1348] (rows=7 width=212)
+          Select Operator [SEL_1359] (rows=7 width=212)
             Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
           <-Reducer 8 [SIMPLE_EDGE] vectorized
-            SHUFFLE [RS_1347]
-              Select Operator [SEL_1346] (rows=7 width=212)
+            SHUFFLE [RS_1358]
+              Select Operator [SEL_1357] (rows=7 width=212)
                 Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
-                Group By Operator [GBY_1345] (rows=7 width=220)
+                Group By Operator [GBY_1356] (rows=7 width=220)
                   Output:["_col0","_col1","_col2","_col3","_col5","_col6"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4
                 <-Union 7 [SIMPLE_EDGE]
                   <-Reducer 16 [CONTAINS]
-                    Reduce Output Operator [RS_1195]
+                    Reduce Output Operator [RS_1204]
                       PartitionCols:_col0, _col1, _col2, _col3, _col4
-                      Group By Operator [GBY_1194] (rows=7 width=220)
+                      Group By Operator [GBY_1203] (rows=7 width=220)
                         Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"],aggregations:["sum(_col4)","sum(_col5)"],keys:_col0, _col1, _col2, _col3, 0L
-                        Top N Key Operator [TNK_1193] (rows=3 width=221)
+                        Top N Key Operator [TNK_1202] (rows=3 width=221)
                           keys:_col0, _col1, _col2, _col3, 0L,sort order:+++++,top n:100
-                          Select Operator [SEL_1191] (rows=1 width=223)
+                          Select Operator [SEL_1200] (rows=1 width=223)
                             Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
-                            Filter Operator [FIL_1190] (rows=1 width=244)
+                            Filter Operator [FIL_1199] (rows=1 width=244)
                               predicate:(_col3 > _col5)
-                              Merge Join Operator [MERGEJOIN_1189] (rows=1 width=244)
+                              Merge Join Operator [MERGEJOIN_1198] (rows=1 width=244)
                                 Conds:(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
                               <-Reducer 15 [CUSTOM_SIMPLE_EDGE] vectorized
-                                PARTITION_ONLY_SHUFFLE [RS_1361]
-                                  Group By Operator [GBY_1360] (rows=1 width=132)
-                                    Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)"],keys:KEY._col0, KEY._col1, KEY._col2
-                                  <-Reducer 14 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_235]
-                                      PartitionCols:_col0, _col1, _col2
-                                      Group By Operator [GBY_234] (rows=1 width=132)
-                                        Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["sum(_col3)","count()"],keys:_col0, _col1, _col2
-                                        Select Operator [SEL_232] (rows=1 width=128)
-                                          Output:["_col0","_col1","_col2","_col3"]
-                                          Merge Join Operator [MERGEJOIN_1162] (rows=1 width=128)
-                                            Conds:RS_229._col1=RS_1330._col0(Inner),Output:["_col2","_col3","_col7","_col8","_col9"]
-                                          <-Map 65 [SIMPLE_EDGE] vectorized
-                                            SHUFFLE [RS_1330]
-                                              PartitionCols:_col0
-                                              Select Operator [SEL_1321] (rows=462000 width=15)
-                                                Output:["_col0","_col1","_col2","_col3"]
-                                                TableScan [TS_81] (rows=462000 width=15)
-                                                  default@item,item,Tbl:COMPLETE,Col:COMPLETE,Output:["i_item_sk","i_brand_id","i_class_id","i_category_id"]
-                                          <-Reducer 13 [ONE_TO_ONE_EDGE]
-                                            FORWARD [RS_229]
-                                              PartitionCols:_col1
-                                              Merge Join Operator [MERGEJOIN_1161] (rows=1 width=120)
-                                                Conds:RS_226._col1=RS_227._col0(Inner),Output:["_col1","_col2","_col3"]
-                                              <-Reducer 12 [SIMPLE_EDGE]
-                                                SHUFFLE [RS_226]
-                                                  PartitionCols:_col1
-                                                  Merge Join Operator [MERGEJOIN_1153] (rows=7790806 width=98)
-                                                    Conds:RS_1355._col0=RS_1297._col0(Inner),Output:["_col1","_col2","_col3"]
-                                                  <-Map 10 [SIMPLE_EDGE] vectorized
-                                                    PARTITION_ONLY_SHUFFLE [RS_1297]
+                                PARTITION_ONLY_SHUFFLE [RS_1373]
+                                  Filter Operator [FIL_1372] (rows=1 width=132)
+                                    predicate:_col3 is not null
+                                    Group By Operator [GBY_1371] (rows=1 width=132)
+                                      Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)"],keys:KEY._col0, KEY._col1, KEY._col2
+                                    <-Reducer 14 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_238]
+                                        PartitionCols:_col0, _col1, _col2
+                                        Group By Operator [GBY_237] (rows=1 width=132)
+                                          Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["sum(_col3)","count()"],keys:_col0, _col1, _col2
+                                          Select Operator [SEL_235] (rows=1 width=128)
+                                            Output:["_col0","_col1","_col2","_col3"]
+                                            Merge Join Operator [MERGEJOIN_1171] (rows=1 width=128)
+                                              Conds:RS_232._col1=RS_1339._col0(Inner),Output:["_col2","_col3","_col7","_col8","_col9"]
+                                            <-Map 65 [SIMPLE_EDGE] vectorized
+                                              SHUFFLE [RS_1339]
+                                                PartitionCols:_col0
+                                                Select Operator [SEL_1330] (rows=462000 width=15)
+                                                  Output:["_col0","_col1","_col2","_col3"]
+                                                  TableScan [TS_81] (rows=462000 width=15)
+                                                    default@item,item,Tbl:COMPLETE,Col:COMPLETE,Output:["i_item_sk","i_brand_id","i_class_id","i_category_id"]
+                                            <-Reducer 13 [ONE_TO_ONE_EDGE]
+                                              FORWARD [RS_232]
+                                                PartitionCols:_col1
+                                                Merge Join Operator [MERGEJOIN_1170] (rows=1 width=120)
+                                                  Conds:RS_229._col1=RS_230._col0(Inner),Output:["_col1","_col2","_col3"]
+                                                <-Reducer 12 [SIMPLE_EDGE]
+                                                  SHUFFLE [RS_229]
+                                                    PartitionCols:_col1
+                                                    Merge Join Operator [MERGEJOIN_1162] (rows=7790806 width=98)
+                                                      Conds:RS_1366._col0=RS_1306._col0(Inner),Output:["_col1","_col2","_col3"]
+                                                    <-Map 10 [SIMPLE_EDGE] vectorized
+                                                      PARTITION_ONLY_SHUFFLE [RS_1306]
+                                                        PartitionCols:_col0
+                                                        Select Operator [SEL_1303] (rows=50 width=4)
+                                                          Output:["_col0"]
+                                                          Filter Operator [FIL_1302] (rows=50 width=12)
+                                                            predicate:((d_moy = 11) and (d_year = 2000))
+                                                            TableScan [TS_3] (rows=73049 width=12)
+                                                              default@date_dim,date_dim,Tbl:COMPLETE,Col:COMPLETE,Output:["d_date_sk","d_year","d_moy"]
+                                                    <-Map 79 [SIMPLE_EDGE] vectorized
+                                                      SHUFFLE [RS_1366]
+                                                        PartitionCols:_col0
+                                                        Select Operator [SEL_1365] (rows=286549727 width=123)
+                                                          Output:["_col0","_col1","_col2","_col3"]
+                                                          Filter Operator [FIL_1364] (rows=286549727 width=123)
+                                                            predicate:((cs_sold_date_sk BETWEEN DynamicValue(RS_227_date_dim_d_date_sk_min) AND DynamicValue(RS_227_date_dim_d_date_sk_max) and in_bloom_filter(cs_sold_date_sk, DynamicValue(RS_227_date_dim_d_date_sk_bloom_filter))) and cs_sold_date_sk is not null)
+                                                            TableScan [TS_143] (rows=287989836 width=123)
+                                                              default@catalog_sales,catalog_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["cs_sold_date_sk","cs_item_sk","cs_quantity","cs_list_price"]
+                                                            <-Reducer 17 [BROADCAST_EDGE] vectorized
+                                                              BROADCAST [RS_1363]
+                                                                Group By Operator [GBY_1362] (rows=1 width=12)
+                                                                  Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
+                                                                <-Map 10 [CUSTOM_SIMPLE_EDGE] vectorized
+                                                                  PARTITION_ONLY_SHUFFLE [RS_1314]
+                                                                    Group By Operator [GBY_1311] (rows=1 width=12)
+                                                                      Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
+                                                                      Select Operator [SEL_1307] (rows=50 width=4)
+                                                                        Output:["_col0"]
+                                                                         Please refer to the previous Select Operator [SEL_1303]
+                                                <-Reducer 34 [SIMPLE_EDGE]
+                                                  SHUFFLE [RS_230]
+                                                    PartitionCols:_col0
+                                                    Merge Join Operator [MERGEJOIN_1169] (rows=724 width=4)
+                                                      Conds:RS_1347._col1, _col2, _col3=RS_1370._col0, _col1, _col2(Inner),Output:["_col0"]
+                                                    <-Map 65 [SIMPLE_EDGE] vectorized
+                                                      SHUFFLE [RS_1347]
+                                                        PartitionCols:_col1, _col2, _col3
+                                                        Select Operator [SEL_1340] (rows=458612 width=15)
+                                                          Output:["_col0","_col1","_col2","_col3"]
+                                                          Filter Operator [FIL_1331] (rows=458612 width=15)
+                                                            predicate:(i_brand_id is not null and i_category_id is not null and i_class_id is not null)
+                                                             Please refer to the previous TableScan [TS_81]
+                                                    <-Reducer 33 [ONE_TO_ONE_EDGE] vectorized
+                                                      FORWARD [RS_1370]
+                                                        PartitionCols:_col0, _col1, _col2
+                                                        Select Operator [SEL_1369] (rows=1 width=12)
+                                                          Output:["_col0","_col1","_col2"]
+                                                          Filter Operator [FIL_1368] (rows=1 width=20)
+                                                            predicate:(_col3 = 3L)
+                                                            Group By Operator [GBY_1367] (rows=121728 width=19)
+                                                              Output:["_col0","_col1","_col2","_col3"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2
+                                                            <-Union 32 [SIMPLE_EDGE]
+                                                              <-Reducer 31 [CONTAINS] vectorized
+                                                                Reduce Output Operator [RS_1424]
+                                                                  PartitionCols:_col0, _col1, _col2
+                                                                  Group By Operator [GBY_1423] (rows=121728 width=19)
+                                                                    Output:["_col0","_col1","_col2","_col3"],aggregations:["count(_col3)"],keys:_col0, _col1, _col2
+                                                                    Group By Operator [GBY_1422] (rows=121728 width=19)
+                                                                      Output:["_col0","_col1","_col2","_col3"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2
+                                                                    <-Reducer 26 [SIMPLE_EDGE]
+                                                                      SHUFFLE [RS_169]
+                                                                        PartitionCols:_col0, _col1, _col2
+                                                                        Group By Operator [GBY_25] (rows=121728 width=19)
+                                                                          Output:["_col0","_col1","_col2","_col3"],aggregations:["count()"],keys:_col4, _col5, _col6
+                                                                          Merge Join Operator [MERGEJOIN_1151] (rows=14628613 width=11)
+                                                                            Conds:RS_21._col1=RS_1344._col0(Inner),Output:["_col4","_col5","_col6"]
+                                                                          <-Map 65 [SIMPLE_EDGE] vectorized
+                                                                            SHUFFLE [RS_1344]
+                                                                              PartitionCols:_col0
+                                                                              Select Operator [SEL_1336] (rows=458612 width=15)
+                                                                                Output:["_col0","_col1","_col2","_col3"]
+                                                                                Filter Operator [FIL_1327] (rows=458612 width=15)
+                                                                                  predicate:(i_brand_id is not null and i_category_id is not null and i_class_id is not null)
+                                                                                   Please refer to the previous TableScan [TS_81]
+                                                                          <-Reducer 25 [SIMPLE_EDGE]
+                                                                            SHUFFLE [RS_21]
+                                                                              PartitionCols:_col1
+                                                                              Merge Join Operator [MERGEJOIN_1150] (rows=14736682 width=4)
+                                                                                Conds:RS_1418._col0=RS_1396._col0(Inner),Output:["_col1"]
+                                                                              <-Map 39 [SIMPLE_EDGE] vectorized
+                                                                                SHUFFLE [RS_1396]
+                                                                                  PartitionCols:_col0
+                                                                                  Select Operator [SEL_1395] (rows=1957 width=4)
+                                                                                    Output:["_col0"]
+                                                                                    Filter Operator [FIL_1394] (rows=1957 width=8)
+                                                                                      predicate:d_year BETWEEN 1999 AND 2001
+                                                                                      TableScan [TS_12] (rows=73049 width=8)
+                                                                                        default@date_dim,d1,Tbl:COMPLETE,Col:COMPLETE,Output:["d_date_sk","d_year"]
+                                                                              <-Map 24 [SIMPLE_EDGE] vectorized
+                                                                                SHUFFLE [RS_1418]
+                                                                                  PartitionCols:_col0
+                                                                                  Select Operator [SEL_1417] (rows=550076554 width=7)
+                                                                                    Output:["_col0","_col1"]
+                                                                                    Filter Operator [FIL_1416] (rows=550076554 width=7)
+                                                                                      predicate:((ss_sold_date_sk BETWEEN DynamicValue(RS_19_d1_d_date_sk_min) AND DynamicValue(RS_19_d1_d_date_sk_max) and in_bloom_filter(ss_sold_date_sk, DynamicValue(RS_19_d1_d_date_sk_bloom_filter))) and ss_sold_date_sk is not null)
+                                                                                      TableScan [TS_9] (rows=575995635 width=7)
+                                                                                        default@store_sales,store_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ss_sold_date_sk","ss_item_sk"]
+                                                                                      <-Reducer 40 [BROADCAST_EDGE] vectorized
+                                                                                        BROADCAST [RS_1415]
+                                                                                          Group By Operator [GBY_1414] (rows=1 width=12)
+                                                                                            Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
+                                                                                          <-Map 39 [CUSTOM_SIMPLE_EDGE] vectorized
+                                                                                            SHUFFLE [RS_1410]
+                                                                                              Group By Operator [GBY_1406] (rows=1 width=12)
+                                                                                                Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
+                                                                                                Select Operator [SEL_1397] (rows=1957 width=4)
+                                                                                                  Output:["_col0"]
+                                                                                                   Please refer to the previous Select Operator [SEL_1395]
+                                                              <-Reducer 44 [CONTAINS] vectorized
+                                                                Reduce Output Operator [RS_1438]
+                                                                  PartitionCols:_col0, _col1, _col2
+                                                                  Group By Operator [GBY_1437] (rows=121728 width=19)
+                                                                    Output:["_col0","_col1","_col2","_col3"],aggregations:["count(_col3)"],keys:_col0, _col1, _col2
+                                                                    Group By Operator [GBY_1436] (rows=121728 width=19)
+                                                                      Output:["_col0","_col1","_col2","_col3"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2
+                                                                    <-Reducer 42 [SIMPLE_EDGE]
+                                                                      SHUFFLE [RS_189]
+                                                                        PartitionCols:_col0, _col1, _col2
+                                                                        Group By Operator [GBY_45] (rows=121728 width=19)
+                                                                          Output:["_col0","_col1","_col2","_col3"],aggregations:["count()"],keys:_col4, _col5, _col6
+                                                                          Merge Join Operator [MERGEJOIN_1153] (rows=7620440 width=11)
+                                                                            Conds:RS_41._col1=RS_1345._col0(Inner),Output:["_col4","_col5","_col6"]
+                                                                          <-Map 65 [SIMPLE_EDGE] vectorized
+                                                                            SHUFFLE [RS_1345]
+                                                                              PartitionCols:_col0
+                                                                              Select Operator [SEL_1337] (rows=458612 width=15)
+                                                                                Output:["_col0","_col1","_col2","_col3"]
+                                                                                Filter Operator [FIL_1328] (rows=458612 width=15)
+                                                                                  predicate:(i_brand_id is not null and i_category_id is not null and i_class_id is not null)
+                                                                                   Please refer to the previous TableScan [TS_81]
+                                                                          <-Reducer 41 [SIMPLE_EDGE]
+                                                                            SHUFFLE [RS_41]
+                                                                              PartitionCols:_col1
+                                                                              Merge Join Operator [MERGEJOIN_1152] (rows=7676736 width=4)
+                                                                                Conds:RS_1432._col0=RS_1398._col0(Inner),Output:["_col1"]
+                                                                              <-Map 39 [SIMPLE_EDGE] vectorized
+                                                                                SHUFFLE [RS_1398]
+                                                                                  PartitionCols:_col0
+                                                                                   Please refer to the previous Select Operator [SEL_1395]
+                                                                              <-Map 63 [SIMPLE_EDGE] vectorized
+                                                                                SHUFFLE [RS_1432]
+                                                                                  PartitionCols:_col0
+                                                                                  Select Operator [SEL_1431] (rows=286549727 width=7)
+                                                                                    Output:["_col0","_col1"]
+                                                                                    Filter Operator [FIL_1430] (rows=286549727 width=7)
+                                                                                      predicate:((cs_sold_date_sk BETWEEN DynamicValue(RS_39_d2_d_date_sk_min) AND DynamicValue(RS_39_d2_d_date_sk_max) and in_bloom_filter(cs_sold_date_sk, DynamicValue(RS_39_d2_d_date_sk_bloom_filter))) and cs_sold_date_sk is not null)
+                                                                                      TableScan [TS_29] (rows=287989836 width=7)
+                                                                                        default@catalog_sales,catalog_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["cs_sold_date_sk","cs_item_sk"]
+                                                                                      <-Reducer 46 [BROADCAST_EDGE] vectorized
+                                                                                        BROADCAST [RS_1429]
+                                                                                          Group By Operator [GBY_1428] (rows=1 width=12)
+                                                                                            Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
+                                                                                          <-Map 39 [CUSTOM_SIMPLE_EDGE] vectorized
+                                                                                            SHUFFLE [RS_1411]
+                                                                                              Group By Operator [GBY_1407] (rows=1 width=12)
+                                                                                                Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
+                                                                                                Select Operator [SEL_1399] (rows=1957 width=4)
+                                                                                                  Output:["_col0"]
+                                                                                                   Please refer to the previous Select Operator [SEL_1395]
+                                                              <-Reducer 50 [CONTAINS] vectorized
+                                                                Reduce Output Operator [RS_1452]
+                                                                  PartitionCols:_col0, _col1, _col2
+                                                                  Group By Operator [GBY_1451] (rows=121728 width=19)
+                                                                    Output:["_col0","_col1","_col2","_col3"],aggregations:["count(_col3)"],keys:_col0, _col1, _col2
+                                                                    Group By Operator [GBY_1450] (rows=121728 width=19)
+                                                                      Output:["_col0","_col1","_col2","_col3"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2
+                                                                    <-Reducer 48 [SIMPLE_EDGE]
+                                                                      SHUFFLE [RS_210]
+                                                                        PartitionCols:_col0, _col1, _col2
+                                                                        Group By Operator [GBY_66] (rows=121728 width=19)
+                                                                          Output:["_col0","_col1","_col2","_col3"],aggregations:["count()"],keys:_col4, _col5, _col6
+                                                                          Merge Join Operator [MERGEJOIN_1155] (rows=3828623 width=11)
+                                                                            Conds:RS_62._col1=RS_1346._col0(Inner),Output:["_col4","_col5","_col6"]
+                                                                          <-Map 65 [SIMPLE_EDGE] vectorized
+                                                                            SHUFFLE [RS_1346]
+                                                                              PartitionCols:_col0
+                                                                              Select Operator [SEL_1338] (rows=458612 width=15)
+                                                                                Output:["_col0","_col1","_col2","_col3"]
+                                                                                Filter Operator [FIL_1329] (rows=458612 width=15)
+                                                                                  predicate:(i_brand_id is not null and i_category_id is not null and i_class_id is not null)
+                                                                                   Please refer to the previous TableScan [TS_81]
+                                                                          <-Reducer 47 [SIMPLE_EDGE]
+                                                                            SHUFFLE [RS_62]
+                                                                              PartitionCols:_col1
+                                                                              Merge Join Operator [MERGEJOIN_1154] (rows=3856907 width=4)
+                                                                                Conds:RS_1446._col0=RS_1400._col0(Inner),Output:["_col1"]
+                                                                              <-Map 39 [SIMPLE_EDGE] vectorized
+                                                                                SHUFFLE [RS_1400]
+                                                                                  PartitionCols:_col0
+                                                                                   Please refer to the previous Select Operator [SEL_1395]
+                                                                              <-Map 64 [SIMPLE_EDGE] vectorized
+                                                                                SHUFFLE [RS_1446]
+                                                                                  PartitionCols:_col0
+                                                                                  Select Operator [SEL_1445] (rows=143966864 width=7)
+                                                                                    Output:["_col0","_col1"]
+                                                                                    Filter Operator [FIL_1444] (rows=143966864 width=7)
+                                                                                      predicate:((ws_sold_date_sk BETWEEN DynamicValue(RS_60_d3_d_date_sk_min) AND DynamicValue(RS_60_d3_d_date_sk_max) and in_bloom_filter(ws_sold_date_sk, DynamicValue(RS_60_d3_d_date_sk_bloom_filter))) and ws_sold_date_sk is not null)
+                                                                                      TableScan [TS_50] (rows=144002668 width=7)
+                                                                                        default@web_sales,web_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ws_sold_date_sk","ws_item_sk"]
+                                                                                      <-Reducer 52 [BROADCAST_EDGE] vectorized
+                                                                                        BROADCAST [RS_1443]
+                                                                                          Group By Operator [GBY_1442] (rows=1 width=12)
+                                                                                            Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
+                                                                                          <-Map 39 [CUSTOM_SIMPLE_EDGE] vectorized
+                                                                                            SHUFFLE [RS_1412]
+                                                                                              Group By Operator [GBY_1408] (rows=1 width=12)
+                                                                                                Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
+                                                                                                Select Operator [SEL_1401] (rows=1957 width=4)
+                                                                                                  Output:["_col0"]
+                                                                                                   Please refer to the previous Select Operator [SEL_1395]
+                              <-Reducer 59 [CUSTOM_SIMPLE_EDGE] vectorized
+                                PARTITION_ONLY_SHUFFLE [RS_1377]
+                                  Select Operator [SEL_1376] (rows=1 width=112)
+                                    Output:["_col0"]
+                                    Filter Operator [FIL_1375] (rows=1 width=120)
+                                      predicate:(_col0 is not null and _col1 is not null)
+                                      Group By Operator [GBY_1374] (rows=1 width=120)
+                                        Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)"]
+                                      <-Union 58 [CUSTOM_SIMPLE_EDGE]
+                                        <-Reducer 57 [CONTAINS]
+                                          Reduce Output Operator [RS_1259]
+                                            Group By Operator [GBY_1258] (rows=1 width=120)
+                                              Output:["_col0","_col1"],aggregations:["sum(_col0)","count(_col0)"]
+                                              Select Operator [SEL_1257] (rows=26270325 width=44)
+                                                Output:["_col0"]
+                                                Select Operator [SEL_1255] (rows=14736682 width=0)
+                                                  Output:["_col0","_col1"]
+                                                  Merge Join Operator [MERGEJOIN_1254] (rows=14736682 width=0)
+                                                    Conds:RS_1461._col0=RS_1404._col0(Inner),Output:["_col1","_col2"]
+                                                  <-Map 39 [SIMPLE_EDGE] vectorized
+                                                    SHUFFLE [RS_1404]
+                                                      PartitionCols:_col0
+                                                       Please refer to the previous Select Operator [SEL_1395]
+                                                  <-Map 66 [SIMPLE_EDGE] vectorized
+                                                    SHUFFLE [RS_1461]
+                                                      PartitionCols:_col0
+                                                      Select Operator [SEL_1459] (rows=550076554 width=114)
+                                                        Output:["_col0","_col1","_col2"]
+                                                        Filter Operator [FIL_1458] (rows=550076554 width=114)
+                                                          predicate:((ss_sold_date_sk BETWEEN DynamicValue(RS_106_date_dim_d_date_sk_min) AND DynamicValue(RS_106_date_dim_d_date_sk_max) and in_bloom_filter(ss_sold_date_sk, DynamicValue(RS_106_date_dim_d_date_sk_bloom_filter))) and ss_sold_date_sk is not null)
+                                                          TableScan [TS_99] (rows=575995635 width=114)
+                                                            default@store_sales,store_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ss_sold_date_sk","ss_quantity","ss_list_price"]
+                                                          <-Reducer 56 [BROADCAST_EDGE] vectorized
+                                                            BROADCAST [RS_1457]
+                                                              Group By Operator [GBY_1456] (rows=1 width=12)
+                                                                Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
+                                                              <-Map 39 [CUSTOM_SIMPLE_EDGE] vectorized
+                                                                SHUFFLE [RS_1413]
+                                                                  Group By Operator [GBY_1409] (rows=1 width=12)
+                                                                    Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
+                                                                    Select Operator [SEL_1403] (rows=1957 width=4)
+                                                                      Output:["_col0"]
+                                                                       Please refer to the previous Select Operator [SEL_1395]
+                                        <-Reducer 69 [CONTAINS]
+                                          Reduce Output Operator [RS_1277]
+                                            Group By Operator [GBY_1276] (rows=1 width=120)
+                                              Output:["_col0","_col1"],aggregations:["sum(_col0)","count(_col0)"]
+                                              Select Operator [SEL_1275] (rows=26270325 width=44)
+                                                Output:["_col0"]
+                                                Select Operator [SEL_1273] (rows=7676736 width=94)
+                                                  Output:["_col0","_col1"]
+                                                  Merge Join Operator [MERGEJOIN_1272] (rows=7676736 width=94)
+                                                    Conds:RS_1476._col0=RS_1467._col0(Inner),Output:["_col1","_col2"]
+                                                  <-Map 71 [SIMPLE_EDGE] vectorized
+                                                    PARTITION_ONLY_SHUFFLE [RS_1467]
                                                       PartitionCols:_col0
-                                                      Select Operator [SEL_1294] (rows=50 width=4)
+                                                      Select Operator [SEL_1464] (rows=1957 width=4)
                                                         Output:["_col0"]
-                                                        Filter Operator [FIL_1293] (rows=50 width=12)
-                                                          predicate:((d_moy = 11) and (d_year = 2000))
-                                                          TableScan [TS_3] (rows=73049 width=12)
-                                                            default@date_dim,date_dim,Tbl:COMPLETE,Col:COMPLETE,Output:["d_date_sk","d_year","d_moy"]
-                                                  <-Map 79 [SIMPLE_EDGE] vectorized
-                                                    SHUFFLE [RS_1355]
+                                                        Filter Operator [FIL_1463] (rows=1957 width=8)
+                                                          predicate:d_year BETWEEN 1998 AND 2000
+                                                          TableScan [TS_112] (rows=73049 width=8)
+                                                            default@date_dim,date_dim,Tbl:COMPLETE,Col:COMPLETE,Output:["d_date_sk","d_year"]
+                                                  <-Map 67 [SIMPLE_EDGE] vectorized
+                                                    SHUFFLE [RS_1476]
                                                       PartitionCols:_col0
-                                                      Select Operator [SEL_1354] (rows=286549727 width=123)
-                                                        Output:["_col0","_col1","_col2","_col3"]
-                                                        Filter Operator [FIL_1353] (rows=286549727 width=123)
-                                                          predicate:((cs_sold_date_sk BETWEEN DynamicValue(RS_224_date_dim_d_date_sk_min) AND DynamicValue(RS_224_date_dim_d_date_sk_max) and in_bloom_filter(cs_sold_date_sk, DynamicValue(RS_224_date_dim_d_date_sk_bloom_filter))) and cs_sold_date_sk is not null)
-                                                          TableScan [TS_140] (rows=287989836 width=123)
-                                                            default@catalog_sales,catalog_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["cs_sold_date_sk","cs_item_sk","cs_quantity","cs_list_price"]
-                                                          <-Reducer 17 [BROADCAST_EDGE] vectorized
-                                                            BROADCAST [RS_1352]
-                                                              Group By Operator [GBY_1351] (rows=1 width=12)
+                                                      Select Operator [SEL_1474] (rows=286549727 width=119)
+                                                        Output:["_col0","_col1","_col2"]
+                                                        Filter Operator [FIL_1473] (rows=286549727 width=119)
+                                                          predicate:((cs_sold_date_sk BETWEEN DynamicValue(RS_116_date_dim_d_date_sk_min) AND DynamicValue(RS_116_date_dim_d_date_sk_max) and in_bloom_filter(cs_sold_date_sk, DynamicValue(RS_116_date_dim_d_date_sk_bloom_filter))) and cs_sold_date_sk is not null)
+                                                          TableScan [TS_109] (rows=287989836 width=119)
+                                                            default@catalog_sales,catalog_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["cs_sold_date_sk","cs_quantity","cs_list_price"]
+                                                          <-Reducer 72 [BROADCAST_EDGE] vectorized
+                                                            BROADCAST [RS_1472]
+                                                              Group By Operator [GBY_1471] (rows=1 width=12)
                                                                 Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
-                                                              <-Map 10 [CUSTOM_SIMPLE_EDGE] vectorized
-                                                                PARTITION_ONLY_SHUFFLE [RS_1305]
-                                                                  Group By Operator [GBY_1302] (rows=1 width=12)
+                                                              <-Map 71 [CUSTOM_SIMPLE_EDGE] vectorized
+                                                                PARTITION_ONLY_SHUFFLE [RS_1470]
+                                                                  Group By Operator [GBY_1469] (rows=1 width=12)
                                                                     Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
-                                                                    Select Operator [SEL_1298] (rows=50 width=4)
+                                                                    Select Operator [SEL_1466] (rows=1957 width=4)
                                                                       Output:["_col0"]
-                                                                       Please refer to the previous Select Operator [SEL_1294]
-                                              <-Reducer 34 [SIMPLE_EDGE]
-                                                SHUFFLE [RS_227]
-                                                  PartitionCols:_col0
-                                                  Merge Join Operator [MERGEJOIN_1160] (rows=724 width=4)
-                                                    Conds:RS_1338._col1, _col2, _col3=RS_1359._col0, _col1, _col2(Inner),Output:["_col0"]
-                                                  <-Map 65 [SIMPLE_EDGE] vectorized
-                                                    SHUFFLE [RS_1338]
-                                                      PartitionCols:_col1, _col2, _col3
-                                                      Select Operator [SEL_1331] (rows=458612 width=15)
-                                                        Output:["_col0","_col1","_col2","_col3"]
-                                                        Filter Operator [FIL_1322] (rows=458612 width=15)
-                                                          predicate:(i_brand_id is not null and i_category_id is not null and i_class_id is not null)
-                                                           Please refer to the previous TableScan [TS_81]
-                                                  <-Reducer 33 [ONE_TO_ONE_EDGE] vectorized
-                                                    FORWARD [RS_1359]
-                                                      PartitionCols:_col0, _col1, _col2
-                                                      Select Operator [SEL_1358] (rows=1 width=12)
+                                                                       Please refer to the previous Select Operator [SEL_1464]
+                                        <-Reducer 75 [CONTAINS]
+                                          Reduce Output Operator [RS_1295]
+                                            Group By Operator [GBY_1294] (rows=1 width=120)
+                                              Output:["_col0","_col1"],aggregations:["sum(_col0)","count(_col0)"]
+                                              Select Operator [SEL_1293] (rows=26270325 width=44)
+                                                Output:["_col0"]
+                                                Select Operator [SEL_1291] (rows=3856907 width=114)
+                                                  Output:["_col0","_col1"]
+                                                  Merge Join Operator [MERGEJOIN_1290] (rows=3856907 width=114)
+                                                    Conds:RS_1491._col0=RS_1482._col0(Inner),Output:["_col1","_col2"]
+                                                  <-Map 77 [SIMPLE_EDGE] vectorized
+                                                    PARTITION_ONLY_SHUFFLE [RS_1482]
+                                                      PartitionCols:_col0
+                                                      Select Operator [SEL_1479] (rows=1957 width=4)
+                                                        Output:["_col0"]
+                                                        Filter Operator [FIL_1478] (rows=1957 width=8)
+                                                          predicate:d_year BETWEEN 1998 AND 2000
+                                                          TableScan [TS_123] (rows=73049 width=8)
+                                                            default@date_dim,date_dim,Tbl:COMPLETE,Col:COMPLETE,Output:["d_date_sk","d_year"]
+                                                  <-Map 73 [SIMPLE_EDGE] vectorized
+                                                    SHUFFLE [RS_1491]
+                                                      PartitionCols:_col0
+                                                      Select Operator [SEL_1489] (rows=143966864 width=119)
                                                         Output:["_col0","_col1","_col2"]
-                                                        Filter Operator [FIL_1357] (rows=1 width=20)
-                                                          predicate:(_col3 = 3L)
-                                                          Group By Operator [GBY_1356] (rows=121728 width=19)
-                                                            Output:["_col0","_col1","_col2","_col3"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2
-                                                          <-Union 32 [SIMPLE_EDGE]
-                                                            <-Reducer 31 [CONTAINS] vectorized
-                                                              Reduce Output Operator [RS_1409]
-                                                                PartitionCols:_col0, _col1, _col2
-                                                                Group By Operator [GBY_1408] (rows=121728 width=19)
-                                                                  Output:["_col0","_col1","_col2","_col3"],aggregations:["count(_col3)"],keys:_col0, _col1, _col2
-                                                                  Group By Operator [GBY_1407] (rows=121728 width=19)
-                                                                    Output:["_col0","_col1","_col2","_col3"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2
-                                                                  <-Reducer 26 [SIMPLE_EDGE]
-                                                                    SHUFFLE [RS_166]
-                                                                      PartitionCols:_col0, _col1, _col2
-                                                                      Group By Operator [GBY_25] (rows=121728 width=19)
-                                                                        Output:["_col0","_col1","_col2","_col3"],aggregations:["count()"],keys:_col4, _col5, _col6
-                                                                        Merge Join Operator [MERGEJOIN_1142] (rows=14628613 width=11)
-                                                                          Conds:RS_21._col1=RS_1335._col0(Inner),Output:["_col4","_col5","_col6"]
-                                                                        <-Map 65 [SIMPLE_EDGE] vectorized
-                                                                          SHUFFLE [RS_1335]
-                                                                            PartitionCols:_col0
-                                                                            Select Operator [SEL_1327] (rows=458612 width=15)
-                                                                              Output:["_col0","_col1","_col2","_col3"]
-                                                                              Filter Operator [FIL_1318] (rows=458612 width=15)
-                                                                                predicate:(i_brand_id is not null and i_category_id is not null and i_class_id is not null)
-                                                                                 Please refer to the previous TableScan [TS_81]
-                                                                        <-Reducer 25 [SIMPLE_EDGE]
-                                                                          SHUFFLE [RS_21]
-                                                                            PartitionCols:_col1
-                                                                            Merge Join Operator [MERGEJOIN_1141] (rows=14736682 width=4)
-                                                                              Conds:RS_1403._col0=RS_1381._col0(Inner),Output:["_col1"]
-                                                                            <-Map 39 [SIMPLE_EDGE] vectorized
-                                                                              SHUFFLE [RS_1381]
-                                                                                PartitionCols:_col0
-                                                                                Select Operator [SEL_1380] (rows=1957 width=4)
-                                                                                  Output:["_col0"]
-                                                                                  Filter Operator [FIL_1379] (rows=1957 width=8)
-                                                                                    predicate:d_year BETWEEN 1999 AND 2001
-                                                                                    TableScan [TS_12] (rows=73049 width=8)
-                                                                                      default@date_dim,d1,Tbl:COMPLETE,Col:COMPLETE,Output:["d_date_sk","d_year"]
-                                                                            <-Map 24 [SIMPLE_EDGE] vectorized
-                                                                              SHUFFLE [RS_1403]
-                                                                                PartitionCols:_col0
-                                                                                Select Operator [SEL_1402] (rows=550076554 width=7)
-                                                                                  Output:["_col0","_col1"]
-                                                                                  Filter Operator [FIL_1401] (rows=550076554 width=7)
-                                                                                    predicate:((ss_sold_date_sk BETWEEN DynamicValue(RS_19_d1_d_date_sk_min) AND DynamicValue(RS_19_d1_d_date_sk_max) and in_bloom_filter(ss_sold_date_sk, DynamicValue(RS_19_d1_d_date_sk_bloom_filter))) and ss_sold_date_sk is not null)
-                                                                                    TableScan [TS_9] (rows=575995635 width=7)
-                                                                                      default@store_sales,store_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ss_sold_date_sk","ss_item_sk"]
-                                                                                    <-Reducer 40 [BROADCAST_EDGE] vectorized
-                                                                                      BROADCAST [RS_1400]
-                                                                                        Group By Operator [GBY_1399] (rows=1 width=12)
-                                                                                          Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
-                                                                                        <-Map 39 [CUSTOM_SIMPLE_EDGE] vectorized
-                                                                                          SHUFFLE [RS_1395]
-                                                                                            Group By Operator [GBY_1391] (rows=1 width=12)
-                                                                                              Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
-                                                                                              Select Operator [SEL_1382] (rows=1957 width=4)
-                                                                                                Output:["_col0"]
-                                                                                                 Please refer to the previous Select Operator [SEL_1380]
-                                                            <-Reducer 44 [CONTAINS] vectorized
-                                                              Reduce Output Operator [RS_1423]
-                                                                PartitionCols:_col0, _col1, _col2
-                                                                Group By Operator [GBY_1422] (rows=121728 width=19)
-                                                                  Output:["_col0","_col1","_col2","_col3"],aggregations:["count(_col3)"],keys:_col0, _col1, _col2
-                                                                  Group By Operator [GBY_1421] (rows=121728 width=19)
-                                                                    Output:["_col0","_col1","_col2","_col3"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2
-                                                                  <-Reducer 42 [SIMPLE_EDGE]
-                                                                    SHUFFLE [RS_186]
-                                                                      PartitionCols:_col0, _col1, _col2
-                                                                      Group By Operator [GBY_45] (rows=121728 width=19)
-                                                                        Output:["_col0","_col1","_col2","_col3"],aggregations:["count()"],keys:_col4, _col5, _col6
-                                                                        Merge Join Operator [MERGEJOIN_1144] (rows=7620440 width=11)
-                                                                          Conds:RS_41._col1=RS_1336._col0(Inner),Output:["_col4","_col5","_col6"]
-                                                                        <-Map 65 [SIMPLE_EDGE] vectorized
-                                                                          SHUFFLE [RS_1336]
-                                                                            PartitionCols:_col0
-                                                                            Select Operator [SEL_1328] (rows=458612 width=15)
-                                                                              Output:["_col0","_col1","_col2","_col3"]
-                                                                              Filter Operator [FIL_1319] (rows=458612 width=15)
-                                                                                predicate:(i_brand_id is not null and i_category_id is not null and i_class_id is not null)
-                                                                                 Please refer to the previous TableScan [TS_81]
-                                                                        <-Reducer 41 [SIMPLE_EDGE]
-                                                                          SHUFFLE [RS_41]
-                                                                            PartitionCols:_col1
-                                                                            Merge Join Operator [MERGEJOIN_1143] (rows=7676736 width=4)
-                                                                              Conds:RS_1417._col0=RS_1383._col0(Inner),Output:["_col1"]
-                                                                            <-Map 39 [SIMPLE_EDGE] vectorized
-                                                                              SHUFFLE [RS_1383]
-                                                                                PartitionCols:_col0
-                                                                                 Please refer to the previous Select Operator [SEL_1380]
-                                                                            <-Map 63 [SIMPLE_EDGE] vectorized
-                                                                              SHUFFLE [RS_1417]
-                                                                                PartitionCols:_col0
-                                                                                Select Operator [SEL_1416] (rows=286549727 width=7)
-                                                                                  Output:["_col0","_col1"]
-                                                                                  Filter Operator [FIL_1415] (rows=286549727 width=7)
-                                                                                    predicate:((cs_sold_date_sk BETWEEN DynamicValue(RS_39_d2_d_date_sk_min) AND DynamicValue(RS_39_d2_d_date_sk_max) and in_bloom_filter(cs_sold_date_sk, DynamicValue(RS_39_d2_d_date_sk_bloom_filter))) and cs_sold_date_sk is not null)
-                                                                                    TableScan [TS_29] (rows=287989836 width=7)
-                                                                                      default@catalog_sales,catalog_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["cs_sold_date_sk","cs_item_sk"]
-                                                                                    <-Reducer 46 [BROADCAST_EDGE] vectorized
-                                                                                      BROADCAST [RS_1414]
-                                                                                        Group By Operator [GBY_1413] (rows=1 width=12)
-                                                                                          Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
-                                                                                        <-Map 39 [CUSTOM_SIMPLE_EDGE] vectorized
-                                                                                          SHUFFLE [RS_1396]
-                                                                                            Group By Operator [GBY_1392] (rows=1 width=12)
-                                                                                              Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
-                                                                                              Select Operator [SEL_1384] (rows=1957 width=4)
-                                                                                                Output:["_col0"]
-                                                                                                 Please refer to the previous Select Operator [SEL_1380]
-                                                            <-Reducer 50 [CONTAINS] vectorized
-                                                              Reduce Output Operator [RS_1437]
-                                                                PartitionCols:_col0, _col1, _col2
-                                                                Group By Operator [GBY_1436] (rows=121728 width=19)
-                                                                  Output:["_col0","_col1","_col2","_col3"],aggregations:["count(_col3)"],keys:_col0, _col1, _col2
-                                                                  Group By Operator [GBY_1435] (rows=121728 width=19)
-                                                                    Output:["_col0","_col1","_col2","_col3"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2
-                                                                  <-Reducer 48 [SIMPLE_EDGE]
-                                                                    SHUFFLE [RS_207]
-                                                                      PartitionCols:_col0, _col1, _col2
-                                                                      Group By Operator [GBY_66] (rows=121728 width=19)
-                                                                        Output:["_col0","_col1","_col2","_col3"],aggregations:["count()"],keys:_col4, _col5, _col6
-                                                                        Merge Join Operator [MERGEJOIN_1146] (rows=3828623 width=11)
-                                                                          Conds:RS_62._col1=RS_1337._col0(Inner),Output:["_col4","_col5","_col6"]
-                                                                        <-Map 65 [SIMPLE_EDGE] vectorized
-                                                                          SHUFFLE [RS_1337]
-                                                                            PartitionCols:_col0
-                                                                            Select Operator [SEL_1329] (rows=458612 width=15)
-                                                                              Output:["_col0","_col1","_col2","_col3"]
-                                                                              Filter Operator [FIL_1320] (rows=458612 width=15)
-                                                                                predicate:(i_brand_id is not null and i_category_id is not null and i_class_id is not null)
-                                                                                 Please refer to the previous TableScan [TS_81]
-                                                                        <-Reducer 47 [SIMPLE_EDGE]
-                                                                          SHUFFLE [RS_62]
-                                                                            PartitionCols:_col1
-                                                                            Merge Join Operator [MERGEJOIN_1145] (rows=3856907 width=4)
-                                                                              Conds:RS_1431._col0=RS_1385._col0(Inner),Output:["_col1"]
-                                                                            <-Map 39 [SIMPLE_EDGE] vectorized
-                                                                              SHUFFLE [RS_1385]
-                                                                                PartitionCols:_col0
-                                                                                 Please refer to the previous Select Operator [SEL_1380]
-                                                                            <-Map 64 [SIMPLE_EDGE] vectorized
-                                                                              SHUFFLE [RS_1431]
-                                                                                PartitionCols:_col0
-                                                                                Select Operator [SEL_1430] (rows=143966864 width=7)
-                                                                                  Output:["_col0","_col1"]
-                                                                                  Filter Operator [FIL_1429] (rows=143966864 width=7)
-                                                                                    predicate:((ws_sold_date_sk BETWEEN DynamicValue(RS_60_d3_d_date_sk_min) AND DynamicValue(RS_60_d3_d_date_sk_max) and in_bloom_filter(ws_sold_date_sk, DynamicValue(RS_60_d3_d_date_sk_bloom_filter))) and ws_sold_date_sk is not null)
-                                                                                    TableScan [TS_50] (rows=144002668 width=7)
-                                                                                      default@web_sales,web_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ws_sold_date_sk","ws_item_sk"]
-                                                                                    <-Reducer 52 [BROADCAST_EDGE] vectorized
-                                                                                      BROADCAST [RS_1428]
-                                                                                        Group By Operator [GBY_1427] (rows=1 width=12)
-                                                                                          Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
-                                                                                        <-Map 39 [CUSTOM_SIMPLE_EDGE] vectorized
-                                                                                          SHUFFLE [RS_1397]
-                                                                                            Group By Operator [GBY_1393] (rows=1 width=12)
-                                                                                              Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
-                                                                                              Select Operator [SEL_1386] (rows=1957 width=4)
-                                                                                                Output:["_col0"]
-                                                                                                 Please refer to the previous Select Operator [SEL_1380]
-                              <-Reducer 59 [CUSTOM_SIMPLE_EDGE] vectorized
-                                PARTITION_ONLY_SHUFFLE [RS_1364]
-                                  Select Operator [SEL_1363] (rows=1 width=112)
-                                    Output:["_col0"]
-                                    Group By Operator [GBY_1362] (rows=1 width=120)
-                                      Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)"]
-                                    <-Union 58 [CUSTOM_SIMPLE_EDGE]
-                                      <-Reducer 57 [CONTAINS]
-                                        Reduce Output Operator [RS_1250]
-                                          Group By Operator [GBY_1249] (rows=1 width=120)
-                                            Output:["_col0","_col1"],aggregations:["sum(_col0)","count(_col0)"]
-                                            Select Operator [SEL_1248] (rows=26270325 width=44)
-                                              Output:["_col0"]
-                                              Select Operator [SEL_1246] (rows=14736682 width=0)
-                                                Output:["_col0","_col1"]
-                                                Merge Join Operator [MERGEJOIN_1245] (rows=14736682 width=0)
-                                                  Conds:RS_1446._col0=RS_1389._col0(Inner),Output:["_col1","_col2"]
-                                                <-Map 39 [SIMPLE_EDGE] vectorized
-                                                  SHUFFLE [RS_1389]
-                                                    PartitionCols:_col0
-                                                     Please refer to the previous Select Operator [SEL_1380]
-                                                <-Map 66 [SIMPLE_EDGE] vectorized
-                                                  SHUFFLE [RS_1446]
-                                                    PartitionCols:_col0
-                                                    Select Operator [SEL_1444] (rows=550076554 width=114)
-                                                      Output:["_col0","_col1","_col2"]
-                                                      Filter Operator [FIL_1443] (rows=550076554 width=114)
-                                                        predicate:((ss_sold_date_sk BETWEEN DynamicValue(RS_105_date_dim_d_date_sk_min) AND DynamicValue(RS_105_date_dim_d_date_sk_max) and in_bloom_filter(ss_sold_date_sk, DynamicValue(RS_105_date_dim_d_date_sk_bloom_filter))) and ss_sold_date_sk is not null)
-                                                        TableScan [TS_98] (rows=575995635 width=114)
-                                                          default@store_sales,store_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ss_sold_date_sk","ss_quantity","ss_list_price"]
-                                                        <-Reducer 56 [BROADCAST_EDGE] vectorized
-                                                          BROADCAST [RS_1442]
-                                                            Group By Operator [GBY_1441] (rows=1 width=12)
-                                                              Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
-                                                            <-Map 39 [CUSTOM_SIMPLE_EDGE] vectorized
-                                                              SHUFFLE [RS_1398]
-                                                                Group By Operator [GBY_1394] (rows=1 width=12)
-                                                                  Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
-                                                                  Select Operator [SEL_1388] (rows=1957 width=4)
-                                                                    Output:["_col0"]
-                                                                     Please refer to the previous Select Operator [SEL_1380]
-                                      <-Reducer 69 [CONTAINS]
-                                        Reduce Output Operator [RS_1268]
-                                          Group By Operator [GBY_1267] (rows=1 width=120)
-                                            Output:["_col0","_col1"],aggregations:["sum(_col0)","count(_col0)"]
-                                            Select Operator [SEL_1266] (rows=26270325 width=44)
-                                              Output:["_col0"]
-                                              Select Operator [SEL_1264] (rows=7676736 width=94)
-                                                Output:["_col0","_col1"]
-                                                Merge Join Operator [MERGEJOIN_1263] (rows=7676736 width=94)
-                                                  Conds:RS_1461._col0=RS_1452._col0(Inner),Output:["_col1","_col2"]
-                                                <-Map 71 [SIMPLE_EDGE] vectorized
-                                                  PARTITION_ONLY_SHUFFLE [RS_1452]
-                                                    PartitionCols:_col0
-                                                    Select Operator [SEL_1449] (rows=1957 width=4)
-                                                      Output:["_col0"]
-                                                      Filter Operator [FIL_1448] (rows=1957 width=8)
-                                                        predicate:d_year BETWEEN 1998 AND 2000
-                                                        TableScan [TS_111] (rows=73049 width=8)
-                                                          default@date_dim,date_dim,Tbl:COMPLETE,Col:COMPLETE,Output:["d_date_sk","d_year"]
-                                                <-Map 67 [SIMPLE_EDGE] vectorized
-                                                  SHUFFLE [RS_1461]
-                                                    PartitionCols:_col0
-                                                    Select Operator [SEL_1459] (rows=286549727 width=119)
-                                                      Output:["_col0","_col1","_col2"]
-                                                      Filter Operator [FIL_1458] (rows=286549727 width=119)
-                                                        predicate:((cs_sold_date_sk BETWEEN DynamicValue(RS_115_date_dim_d_date_sk_min) AND DynamicValue(RS_115_date_dim_d_date_sk_max) and in_bloom_filter(cs_sold_date_sk, DynamicValue(RS_115_date_dim_d_date_sk_bloom_filter))) and cs_sold_date_sk is not null)
-                                                        TableScan [TS_108] (rows=287989836 width=119)
-                                                          default@catalog_sales,catalog_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["cs_sold_date_sk","cs_quantity","cs_list_price"]
-                                                        <-Reducer 72 [BROADCAST_EDGE] vectorized
-                                                          BROADCAST [RS_1457]
-                                                            Group By Operator [GBY_1456] (rows=1 width=12)
-                                                              Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
-                                                            <-Map 71 [CUSTOM_SIMPLE_EDGE] vectorized
-                                                              PARTITION_ONLY_SHUFFLE [RS_1455]
-                                                                Group By Operator [GBY_1454] (rows=1 width=12)
-                                                                  Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
-                                                                  Select Operator [SEL_1451] (rows=1957 width=4)
-                                                                    Output:["_col0"]
-                                                                     Please refer to the previous Select Operator [SEL_1449]
-                                      <-Reducer 75 [CONTAINS]
-                                        Reduce Output Operator [RS_1286]
-                                          Group By Operator [GBY_1285] (rows=1 width=120)
-                                            Output:["_col0","_col1"],aggregations:["sum(_col0)","count(_col0)"]
-                                            Select Operator [SEL_1284] (rows=26270325 width=44)
-                                              Output:["_col0"]
-                                              Select Operator [SEL_1282] (rows=3856907 width=114)
-                                                Output:["_col0","_col1"]
-                                                Merge Join Operator [MERGEJOIN_1281] (rows=3856907 width=114)
-                                                  Conds:RS_1476._col0=RS_1467._col0(Inner),Output:["_col1","_col2"]
-                                                <-Map 77 [SIMPLE_EDGE] vectorized
-                                                  PARTITION_ONLY_SHUFFLE [RS_1467]
-                                                    PartitionCols:_col0
-                                                    Select Operator [SEL_1464] (rows=1957 width=4)
-                                                      Output:["_col0"]
-                                                      Filter Operator [FIL_1463] (rows=1957 width=8)
-                                                        predicate:d_year BETWEEN 1998 AND 2000
-                                                        TableScan [TS_122] (rows=73049 width=8)
-                                                          default@date_dim,date_dim,Tbl:COMPLETE,Col:COMPLETE,Output:["d_date_sk","d_year"]
-                                                <-Map 73 [SIMPLE_EDGE] vectorized
-                                                  SHUFFLE [RS_1476]
-                                                    PartitionCols:_col0
-                                                    Select Operator [SEL_1474] (rows=143966864 width=119)
-                                                      Output:["_col0","_col1","_col2"]
-                                                      Filter Operator [FIL_1473] (rows=143966864 width=119)
-                                                        predicate:((ws_sold_date_sk BETWEEN DynamicValue(RS_126_date_dim_d_date_sk_min) AND DynamicValue(RS_126_date_dim_d_date_sk_max) and in_bloom_filter(ws_sold_date_sk, DynamicValue(RS_126_date_dim_d_date_sk_bloom_filter))) and ws_sold_date_sk is not null)
-                                                        TableScan [TS_119] (rows=144002668 width=119)
-                                                          default@web_sales,web_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ws_sold_date_sk","ws_quantity","ws_list_price"]
-                                                        <-Reducer 78 [BROADCAST_EDGE] vectorized
-                                                          BROADCAST [RS_1472]
-                                                            Group By Operator [GBY_1471] (rows=1 width=12)
-                                                              Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
-                                                            <-Map 77 [CUSTOM_SIMPLE_EDGE] vectorized
-                                                              PARTITION_ONLY_SHUFFLE [RS_1470]
-                                                                Group By Operator [GBY_1469] (rows=1 width=12)
-                                                                  Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
-                                                                  Select Operator [SEL_1466] (rows=1957 width=4)
-                                                                    Output:["_col0"]
-                                                                     Please refer to the previous Select Operator [SEL_1464]
+                                                        Filter Operator [FIL_1488] (rows=143966864 width=119)
+                                                          predicate:((ws_sold_date_sk BETWEEN DynamicValue(RS_127_date_dim_d_date_sk_min) AND DynamicValue(RS_127_date_dim_d_date_sk_max) and in_bloom_filter(ws_sold_date_sk, DynamicValue(RS_127_date_dim_d_date_sk_bloom_filter))) and ws_sold_date_sk is not null)
+                                                          TableScan [TS_120] (rows=144002668 width=119)
+                                                            default@web_sales,web_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ws_sold_date_sk","ws_quantity","ws_list_price"]
+                                                          <-Reducer 78 [BROADCAST_EDGE] vectorized
+                                                            BROADCAST [RS_1487]
+                                                              Group By Operator [GBY_1486] (rows=1 width=12)
+                                                                Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
+                                                              <-Map 77 [CUSTOM_SIMPLE_EDGE] vectorized
+                                                                PARTITION_ONLY_SHUFFLE [RS_1485]
+                                                                  Group By Operator [GBY_1484] (rows=1 width=12)
+                                                                    Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
+                                                                    Select Operator [SEL_1481] (rows=1957 width=4)
+                                                                      Output:["_col0"]
+                                                                       Please refer to the previous Select Operator [SEL_1479]
                   <-Reducer 22 [CONTAINS]
-                    Reduce Output Operator [RS_1202]
+                    Reduce Output Operator [RS_1211]
                       PartitionCols:_col0, _col1, _col2, _col3, _col4
-                      Group By Operator [GBY_1201] (rows=7 width=220)
+                      Group By Operator [GBY_1210] (rows=7 width=220)
                         Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"],aggregations:["sum(_col4)","sum(_col5)"],keys:_col0, _col1, _col2, _col3, 0L
-                        Top N Key Operator [TNK_1200] (rows=3 width=221)
+                        Top N Key Operator [TNK_1209] (rows=3 width=221)
                           keys:_col0, _col1, _col2, _col3, 0L,sort order:+++++,top n:100
-                          Select Operator [SEL_1198] (rows=1 width=219)
+                          Select Operator [SEL_1207] (rows=1 width=219)
                             Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
-                            Filter Operator [FIL_1197] (rows=1 width=244)
+                            Filter Operator [FIL_1206] (rows=1 width=244)
                               predicate:(_col3 > _col5)
-                              Merge Join Operator [MERGEJOIN_1196] (rows=1 width=244)
+                              Merge Join Operator [MERGEJOIN_1205] (rows=1 width=244)
                                 Conds:(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
                               <-Reducer 21 [CUSTOM_SIMPLE_EDGE] vectorized
-                                PARTITION_ONLY_SHUFFLE [RS_1375]
-                                  Group By Operator [GBY_1374] (rows=1 width=132)
-                                    Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)"],keys:KEY._col0, KEY._col1, KEY._col2
-                                  <-Reducer 20 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_376]
-                                      PartitionCols:_col0, _col1, _col2
-                                      Group By Operator [GBY_375] (rows=1 width=132)
-                                        Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["sum(_col3)","count()"],keys:_col0, _col1, _col2
-                                        Select Operator [SEL_373] (rows=1 width=128)
-                                          Output:["_col0","_col1","_col2","_col3"]
-                                          Merge Join Operator [MERGEJOIN_1175] (rows=1 width=128)
-                                            Conds:RS_370._col1=RS_1332._col0(Inner),Output:["_col2","_col3","_col7","_col8","_col9"]
-                                          <-Map 65 [SIMPLE_EDGE] vectorized
-                                            SHUFFLE [RS_1332]
-                                              PartitionCols:_col0
-                                              Select Operator [SEL_1323] (rows=462000 width=15)
-                                                Output:["_col0","_col1","_col2","_col3"]
-                                                 Please refer to the previous TableScan [TS_81]
-                                          <-Reducer 19 [ONE_TO_ONE_EDGE]
-                                            FORWARD [RS_370]
-                                              PartitionCols:_col1
-                                              Merge Join Operator [MERGEJOIN_1174] (rows=1 width=120)
-                                                Conds:RS_367._col1=RS_368._col0(Inner),Output:["_col1","_col2","_col3"]
-                                              <-Reducer 18 [SIMPLE_EDGE]
-                                                SHUFFLE [RS_367]
-                                                  PartitionCols:_col1
-                                                  Merge Join Operator [MERGEJOIN_1166] (rows=3942084 width=118)
-                                                    Conds:RS_1369._col0=RS_1299._col0(Inner),Output:["_col1","_col2","_col3"]
-                                                  <-Map 10 [SIMPLE_EDGE] vectorized
-                                                    PARTITION_ONLY_SHUFFLE [RS_1299]
-                                                      PartitionCols:_col0
-                                                       Please refer to the previous Select Operator [SEL_1294]
-                                                  <-Map 80 [SIMPLE_EDGE] vectorized
-                                                    SHUFFLE [RS_1369]
-                                                      PartitionCols:_col0
-                                                      Select Operator [SEL_1368] (rows=143966864 width=123)
-                                                        Output:["_col0","_col1","_col2","_col3"]
-                                                        Filter Operator [FIL_1367] (rows=143966864 width=123)
-                                                          predicate:((ws_sold_date_sk BETWEEN DynamicValue(RS_365_date_dim_d_date_sk_min) AND DynamicValue(RS_365_date_dim_d_date_sk_max) and in_bloom_filter(ws_sold_date_sk, DynamicValue(RS_365_date_dim_d_date_sk_bloom_filter))) and ws_sold_date_sk is not null)
-                                                          TableScan [TS_281] (rows=144002668 width=123)
-                                                            default@web_sales,web_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ws_sold_date_sk","ws_item_sk","ws_quantity","ws_list_price"]
-                                                          <-Reducer 23 [BROADCAST_EDGE] vectorized
-                                                            BROADCAST [RS_1366]
-                                                              Group By Operator [GBY_1365] (rows=1 width=12)
-                                                                Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
-                                                              <-Map 10 [CUSTOM_SIMPLE_EDGE] vectorized
-                                                                PARTITION_ONLY_SHUFFLE [RS_1306]
-                                                                  Group By Operator [GBY_1303] (rows=1 width=12)
-                                                                    Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
-                                                                    Select Operator [SEL_1300] (rows=50 width=4)
-                                                                      Output:["_col0"]
-                                                                       Please refer to the previous Select Operator [SEL_1294]
-                                              <-Reducer 38 [SIMPLE_EDGE]
-                                                SHUFFLE [RS_368]
-                                                  PartitionCols:_col0
-                                                  Merge Join Operator [MERGEJOIN_1173] (rows=724 width=4)
-                                                    Conds:RS_1339._col1, _col2, _col3=RS_1373._col0, _col1, _col2(Inner),Output:["_col0"]
-                                                  <-Map 65 [SIMPLE_EDGE] vectorized
-                                                    SHUFFLE [RS_1339]
-                                                      PartitionCols:_col1, _col2, _col3
-                                                      Select Operator [SEL_1333] (rows=458612 width=15)
-                                                        Output:["_col0","_col1","_col2","_col3"]
-                                                        Filter Operator [FIL_1324] (rows=458612 width=15)
-                                                          predicate:(i_brand_id is not null and i_category_id is not null and i_class_id is not null)
-                                                           Please refer to the previous TableScan [TS_81]
-                                                  <-Reducer 37 [ONE_TO_ONE_EDGE] vectorized
-                                                    FORWARD [RS_1373]
-                                                      PartitionCols:_col0, _col1, _col2
-                                                      Select Operator [SEL_1372] (rows=1 width=12)
-                                                        Output:["_col0","_col1","_col2"]
-                                                        Filter Operator [FIL_1371] (rows=1 width=20)
-                                                          predicate:(_col3 = 3L)
-                                                          Group By Operator [GBY_1370] (rows=121728 width=19)
-                                                            Output:["_col0","_col1","_col2","_col3"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2
-                                                          <-Union 36 [SIMPLE_EDGE]
-                                                            <-Reducer 35 [CONTAINS] vectorized
-                                                              Reduce Output Operator [RS_1412]
-                                                                PartitionCols:_col0, _col1, _col2
-                                                                Group By Operator [GBY_1411] (rows=121728 width=19)
-                                                                  Output:["_col0","_col1","_col2","_col3"],aggregations:["count(_col3)"],keys:_col0, _col1, _col2
-                                                                  Group By Operator [GBY_1410] (rows=121728 width=19)
-                                                                    Output:["_col0","_col1","_col2","_col3"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2
-                                                                  <-Reducer 26 [SIMPLE_EDGE]
-                                                                    SHUFFLE [RS_307]
-                                                                      PartitionCols:_col0, _col1, _col2
-                                                                       Please refer to the previous Group By Operator [GBY_25]
-                                                            <-Reducer 45 [CONTAINS] vectorized
-                                                              Reduce Output Operator [RS_1426]
-                                                                PartitionCols:_col0, _col1, _col2
-                                                                Group By Operator [GBY_1425] (rows=121728 width=19)
-                                                                  Output:["_col0","_col1","_col2","_col3"],aggregations:["count(_col3)"],keys:_col0, _col1, _col2
-                                                                  Group By Operator [GBY_1424] (rows=121728 width=19)
-                                                                    Output:["_col0","_col1","_col2","_col3"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2
-                                                                  <-Reducer 42 [SIMPLE_EDGE]
-                                                                    SHUFFLE [RS_327]
-                                                                      PartitionCols:_col0, _col1, _col2
-                                                                       Please refer to the previous Group By Operator [GBY_45]
-                                                            <-Reducer 51 [CONTAINS] vectorized
-                                                              Reduce Output Operator [RS_1440]
-                                                                PartitionCols:_col0, _col1, _col2
-                                                                Group By Operator [GBY_1439] (rows=121728 width=19)
-                                                                  Output:["_col0","_col1","_col2","_col3"],aggregations:["count(_col3)"],keys:_col0, _col1, _col2
-                                                                  Group By Operator [GBY_1438] (rows=121728 width=19)
-                                                                    Output:["_col0","_col1","_col2","_col3"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2
-                                                                  <-Reducer 48 [SIMPLE_EDGE]
-                                                                    SHUFFLE [RS_348]
-                                                                      PartitionCols:_col0, _col1, _col2
-                                                                       Please refer to the previous Group By Operator [GBY_66]
+                                PARTITION_ONLY_SHUFFLE [RS_1389]
+                                  Filter Operator [FIL_1388] (rows=1 width=132)
+                                    predicate:_col3 is not null
+                                    Group By Operator [GBY_1387] (rows=1 width=132)
+                                      Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)"],keys:KEY._col0, KEY._col1, KEY._col2
+                                    <-Reducer 20 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_382]
+                                        PartitionCols:_col0, _col1, _col2
+                                        Group By Operator [GBY_381] (rows=1 width=132)
+                                          Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["sum(_col3)","count()"],keys:_col0, _col1, _col2
+                                          Select Operator [SEL_379] (rows=1 width=128)
+                                            Output:["_col0","_col1","_col2","_col3"]
+                                            Merge Join Operator [MERGEJOIN_1184] (rows=1 width=128)
+                                              Conds:RS_376._col1=RS_1341._col0(Inner),Output:["_col2","_col3","_col7","_col8","_col9"]
+                                            <-Map 65 [SIMPLE_EDGE] vectorized
+                                              SHUFFLE [RS_1341]
+                                                PartitionCols:_col0
+                                                Select Operator [SEL_1332] (rows=462000 width=15)
+                                                  Output:["_col0","_col1","_col2","_col3"]
+                                                   Please refer to the previous TableScan [TS_81]
+                                            <-Reducer 19 [ONE_TO_ONE_EDGE]
+                                              FORWARD [RS_376]
+                                                PartitionCols:_col1
+                                                Merge Join Operator [MERGEJOIN_1183] (rows=1 width=120)
+                                                  Conds:RS_373._col1=RS_374._col0(Inner),Output:["_col1","_col2","_col3"]
+                                                <-Reducer 18 [SIMPLE_EDGE]
+                                                  SHUFFLE [RS_373]
+                                                    PartitionCols:_col1
+                                                    Merge Join Operator [MERGEJOIN_1175] (rows=3942084 width=118)
+                                                      Conds:RS_1382._col0=RS_1308._col0(Inner),Output:["_col1","_col2","_col3"]
+                                                    <-Map 10 [SIMPLE_EDGE] vectorized
+                                                      PARTITION_ONLY_SHUFFLE [RS_1308]
+                                                        PartitionCols:_col0
+                                                         Please refer to the previous Select Operator [SEL_1303]
+                                                    <-Map 80 [SIMPLE_EDGE] vectorized
+                                                      SHUFFLE [RS_1382]
+                                                        PartitionCols:_col0
+                                                        Select Operator [SEL_1381] (rows=143966864 width=123)
+                                                          Output:["_col0","_col1","_col2","_col3"]
+                                                          Filter Operator [FIL_1380] (rows=143966864 width=123)
+                                                            predicate:((ws_sold_date_sk BETWEEN DynamicValue(RS_371_date_dim_d_date_sk_min) AND DynamicValue(RS_371_date_dim_d_date_sk_max) and in_bloom_filter(ws_sold_date_sk, DynamicValue(RS_371_date_dim_d_date_sk_bloom_filter))) and ws_sold_date_sk is not null)
+                                                            TableScan [TS_287] (rows=144002668 width=123)
+                                                              default@web_sales,web_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ws_sold_date_sk","ws_item_sk","ws_quantity","ws_list_price"]
+                                                            <-Reducer 23 [BROADCAST_EDGE] vectorized
+                                                              BROADCAST [RS_1379]
+                                                                Group By Operator [GBY_1378] (rows=1 width=12)
+                                                                  Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
+                                                                <-Map 10 [CUSTOM_SIMPLE_EDGE] vectorized
+                                                                  PARTITION_ONLY_SHUFFLE [RS_1315]
+                                                                    Group By Operator [GBY_1312] (rows=1 width=12)
+                                                                      Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
+                                                                      Select Operator [SEL_1309] (rows=50 width=4)
+                                                                        Output:["_col0"]
+                                                                         Please refer to the previous Select Operator [SEL_1303]
+                                                <-Reducer 38 [SIMPLE_EDGE]
+                                                  SHUFFLE [RS_374]
+                                                    PartitionCols:_col0
+                                                    Merge Join Operator [MERGEJOIN_1182] (rows=724 width=4)
+                                                      Conds:RS_1348._col1, _col2, _col3=RS_1386._col0, _col1, _col2(Inner),Output:["_col0"]
+                                                    <-Map 65 [SIMPLE_EDGE] vectorized
+                                                      SHUFFLE [RS_1348]
+                                                        PartitionCols:_col1, _col2, _col3
+                                                        Select Operator [SEL_1342] (rows=458612 width=15)
+                                                          Output:["_col0","_col1","_col2","_col3"]
+                                                          Filter Operator [FIL_1333] (rows=458612 width=15)
+                                                            predicate:(i_brand_id is not null and i_category_id is not null and i_class_id is not null)
+                                                             Please refer to the previous TableScan [TS_81]
+                                                    <-Reducer 37 [ONE_TO_ONE_EDGE] vectorized
+                                                      FORWARD [RS_1386]
+                                                        PartitionCols:_col0, _col1, _col2
+                                                        Select Operator [SEL_1385] (rows=1 width=12)
+                                                          Output:["_col0","_col1","_col2"]
+                                                          Filter Operator [FIL_1384] (rows=1 width=20)
+                                                            predicate:(_col3 = 3L)
+                                                            Group By Operator [GBY_1383] (rows=121728 width=19)
+                                                              Output:["_col0","_col1","_col2","_col3"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2
+                                                            <-Union 36 [SIMPLE_EDGE]
+                                                              <-Reducer 35 [CONTAINS] vectorized
+                                                                Reduce Output Operator [RS_1427]
+                                                                  PartitionCols:_col0, _col1, _col2
+                                                                  Group By Operator [GBY_1426] (rows=121728 width=19)
+                                                                    Output:["_col0","_col1","_col2","_col3"],aggregations:["count(_col3)"],keys:_col0, _col1, _col2
+                                                                    Group By Operator [GBY_1425] (rows=121728 width=19)
+                                                                      Output:["_col0","_col1","_col2","_col3"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2
+                                                                    <-Reducer 26 [SIMPLE_EDGE]
+                                                                      SHUFFLE [RS_313]
+                                                                        PartitionCols:_col0, _col1, _col2
+                                                                         Please refer to the previous Group By Operator [GBY_25]
+                                                              <-Reducer 45 [CONTAINS] vectorized
+                                                                Reduce Output Operator [RS_1441]
+                                                                  PartitionCols:_col0, _col1, _col2
+                                                                  Group By Operator [GBY_1440] (rows=121728 width=19)
+                                                                    Output:["_col0","_col1","_col2","_col3"],aggregations:["count(_col3)"],keys:_col0, _col1, _col2
+                                                                    Group By Operator [GBY_1439] (rows=121728 width=19)
+                                                                      Output:["_col0","_col1","_col2","_col3"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2
+                                                                    <-Reducer 42 [SIMPLE_EDGE]
+                                                                      SHUFFLE [RS_333]
+                                                                        PartitionCols:_col0, _col1, _col2
+                                                                         Please refer to the previous Group By Operator [GBY_45]
+                                                              <-Reducer 51 [CONTAINS] vectorized
+                                                                Reduce Output Operator [RS_1455]
+                                                                  PartitionCols:_col0, _col1, _col2
+                                                                  Group By Operator [GBY_1454] (rows=121728 width=19)
+                                                                    Output:["_col0","_col1","_col2","_col3"],aggregations:["count(_col3)"],keys:_col0, _col1, _col2
+                                                                    Group By Operator [GBY_1453] (rows=121728 width=19)
+                                                                      Output:["_col0","_col1","_col2","_col3"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2
+                                                                    <-Reducer 48 [SIMPLE_EDGE]
+                                                                      SHUFFLE [RS_354]
+                                                                        PartitionCols:_col0, _col1, _col2
+                                                                         Please refer to the previous Group By Operator [GBY_66]
                               <-Reducer 62 [CUSTOM_SIMPLE_EDGE] vectorized
-                                PARTITION_ONLY_SHUFFLE [RS_1378]
-                                  Select Operator [SEL_1377] (rows=1 width=112)
+                                PARTITION_ONLY_SHUFFLE [RS_1393]
+                                  Select Operator [SEL_1392] (rows=1 width=112)
                                     Output:["_col0"]
-                                    Group By Operator [GBY_1376] (rows=1 width=120)
-                                      Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)"]
-                                    <-Union 61 [CUSTOM_SIMPLE_EDGE]
-                                      <-Reducer 60 [CONTAINS]
-                                        Reduce Output Operator [RS_1256]
-                                          Group By Operator [GBY_1255] (rows=1 width=120)
-                                            Output:["_col0","_col1"],aggregations:["sum(_col0)","count(_col0)"]
-                                            Select Operator [SEL_1254] (rows=26270325 width=44)
-                                              Output:["_col0"]
-                                              Select Operator [SEL_1252] (rows=14736682 width=0)
-                                                Output:["_col0","_col1"]
-                                                Merge Join Operator [MERGEJOIN_1251] (rows=14736682 width=0)
-                                                  Conds:RS_1447._col0=RS_1390._col0(Inner),Output:["_col1","_col2"]
-                                                <-Map 39 [SIMPLE_EDGE] vectorized
-                                                  SHUFFLE [RS_1390]
-                                                    PartitionCols:_col0
-                                                     Please refer to the previous Select Operator [SEL_1380]
-                                                <-Map 66 [SIMPLE_EDGE] vectorized
-                                                  SHUFFLE [RS_1447]
-                                                    PartitionCols:_col0
-                                                     Please refer to the previous Select Operator [SEL_1444]
-                                      <-Reducer 70 [CONTAINS]
-                                        Reduce Output Operator [RS_1274]
-                                          Group By Operator [GBY_1273] (rows=1 width=120)
-                                            Output:["_col0","_col1"],aggregations:["sum(_col0)","count(_col0)"]
-                                            Select Operator [SEL_1272] (rows=26270325 width=44)
-                                              Output:["_col0"]
-                                              Select Operator [SEL_1270] (rows=7676736 width=94)
-                                                Output:["_col0","_col1"]
-                                                Merge Join Operator [MERGEJOIN_1269] (rows=7676736 width=94)
-                                                  Conds:RS_1462._col0=RS_1453._col0(Inner),Output:["_col1","_col2"]
-                                                <-Map 71 [SIMPLE_EDGE] vectorized
-                                                  PARTITION_ONLY_SHUFFLE [RS_1453]
-                                                    PartitionCols:_col0
-                                                     Please refer to the previous Select Operator [SEL_1449]
-                                                <-Map 67 [SIMPLE_EDGE] vectorized
-                                                  SHUFFLE [RS_1462]
-                                                    PartitionCols:_col0
-                                                     Please refer to the previous Select Operator [SEL_1459]
-                                      <-Reducer 76 [CONTAINS]
-                                        Reduce Output Operator [RS_1292]
-                                          Group By Operator [GBY_1291] (rows=1 width=120)
-                                            Output:["_col0","_col1"],aggregations:["sum(_col0)","count(_col0)"]
-                                            Select Operator [SEL_1290] (rows=26270325 width=44)
-                                              Output:["_col0"]
-                                              Select Operator [SEL_1288] (rows=3856907 width=114)
-                                                Output:["_col0","_col1"]
-                                                Merge Join Operator [MERGEJOIN_1287] (rows=3856907 width=114)
-                                                  Conds:RS_1477._col0=RS_1468._col0(Inner),Output:["_col1","_col2"]
-                                                <-Map 77 [SIMPLE_EDGE] vectorized
-                                                  PARTITION_ONLY_SHUFFLE [RS_1468]
-                                                    PartitionCols:_col0
-                                                     Please refer to the previous Select Operator [SEL_1464]
-                                                <-Map 73 [SIMPLE_EDGE] vectorized
-                                                  SHUFFLE [RS_1477]
-                                                    PartitionCols:_col0
-                                                     Please refer to the previous Select Operator [SEL_1474]
+                                    Filter Operator [FIL_1391] (rows=1 width=120)
+                                      predicate:(_col0 is not null and _col1 is not null)
+                                      Group By Operator [GBY_1390] (rows=1 width=120)
+                                        Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)"]
+                                      <-Union 61 [CUSTOM_SIMPLE_EDGE]
+                                        <-Reducer 60 [CONTAINS]
+                                          Reduce Output Operator [RS_1265]
+                                            Group By Operator [GBY_1264] (rows=1 width=120)
+                                              Output:["_col0","_col1"],aggregations:["sum(_col0)","count(_col0)"]
+                                              Select Operator [SEL_1263] (rows=26270325 width=44)
+                                                Output:["_col0"]
+                                                Select Operator [SEL_1261] (rows=14736682 width=0)
+                                                  Output:["_col0","_col1"]
+                                                  Merge Join Operator [MERGEJOIN_1260] (rows=14736682 width=0)
+                                                    Conds:RS_1462._col0=RS_1405._col0(Inner),Output:["_col1","_col2"]
+                                                  <-Map 39 [SIMPLE_EDGE] vectorized
+                                                    SHUFFLE [RS_1405]
+                                                      PartitionCols:_col0
+                                                       Please refer to the previous Select Operator [SEL_1395]
+                                                  <-Map 66 [SIMPLE_EDGE] vectorized
+                                                    SHUFFLE [RS_1462]
+                                                      PartitionCols:_col0
+                                                       Please refer to the previous Select Operator [SEL_1459]
+                                        <-Reducer 70 [CONTAINS]
+                                          Reduce Output Operator [RS_1283]
+                                            Group By Operator [GBY_1282] (rows=1 width=120)
+                                              Output:["_col0","_col1"],aggregations:["sum(_col0)","count(_col0)"]
+                                              Select Operator [SEL_1281] (rows=26270325 width=44)
+                                                Output:["_col0"]
+                                                Select Operator [SEL_1279] (rows=7676736 width=94)
+                                                  Output:["_col0","_col1"]
+                                                  Merge Join Operator [MERGEJOIN_1278] (rows=7676736 width=94)
+                                                    Conds:RS_1477._col0=RS_1468._col0(Inner),Output:["_col1","_col2"]
+                                                  <-Map 71 [SIMPLE_EDGE] vectorized
+                                                    PARTITION_ONLY_SHUFFLE [RS_1468]
+                                                      PartitionCols:_col0
+                                                       Please refer to the previous Select Operator [SEL_1464]
+                                                  <-Map 67 [SIMPLE_EDGE] vectorized
+                                                    SHUFFLE [RS_1477]
+                                                      PartitionCols:_col0
+                                                       Please refer to the previous Select Operator [SEL_1474]
+                                        <-Reducer 76 [CONTAINS]
+                                          Reduce Output Operator [RS_1301]
+                                            Group By Operator [GBY_1300] (rows=1 width=120)
+                                              Output:["_col0","_col1"],aggregations:["sum(_col0)","count(_col0)"]
+                                              Select Operator [SEL_1299] (rows=26270325 width=44)
+                                                Output:["_col0"]
+                                                Select Operator [SEL_1297] (rows=3856907 width=114)
+                                                  Output:["_col0","_col1"]
+                                                  Merge Join Operator [MERGEJOIN_1296] (rows=3856907 width=114)
+                                                    Conds:RS_1492._col0=RS_1483._col0(Inner),Output:["_col1","_col2"]
+                                                  <-Map 77 [SIMPLE_EDGE] vectorized
+                                                    PARTITION_ONLY_SHUFFLE [RS_1483]
+                                                      PartitionCols:_col0
+                                                       Please refer to the previous Select Operator [SEL_1479]
+                                                  <-Map 73 [SIMPLE_EDGE] vectorized
+                                                    SHUFFLE [RS_1492]
+                                                      PartitionCols:_col0
+                                                       Please refer to the previous Select Operator [SEL_1489]
                   <-Reducer 6 [CONTAINS]
-                    Reduce Output Operator [RS_1188]
+                    Reduce Output Operator [RS_1197]
                       PartitionCols:_col0, _col1, _col2, _col3, _col4
-                      Group By Operator [GBY_1187] (rows=7 width=220)
+                      Group By Operator [GBY_1196] (rows=7 width=220)
                         Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"],aggregations:["sum(_col4)","sum(_col5)"],keys:_col0, _col1, _col2, _col3, 0L
-                        Top N Key Operator [TNK_1186] (rows=3 width=221)
+                        Top N Key Operator [TNK_1195] (rows=3 width=221)
                           keys:_col0, _col1, _col2, _col3, 0L,sort order:+++++,top n:100
-                          Select Operator [SEL_1184] (rows=1 width=221)
+                          Select Operator [SEL_1193] (rows=1 width=221)
                             Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
-                            Filter Operator [FIL_1183] (rows=1 width=244)
+                            Filter Operator [FIL_1192] (rows=1 width=244)
                               predicate:(_col3 > _col5)
-                              Merge Join Operator [MERGEJOIN_1182] (rows=1 width=244)
+                              Merge Join Operator [MERGEJOIN_1191] (rows=1 width=244)
                                 Conds:(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
                               <-Reducer 5 [CUSTOM_SIMPLE_EDGE] vectorized
-                                PARTITION_ONLY_SHUFFLE [RS_1341]
-                                  Group By Operator [GBY_1340] (rows=1 width=132)
-                                    Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)"],keys:KEY._col0, KEY._col1, KEY._col2
... 12127 lines suppressed ...